From 23603dea28cfe42d15b0e0979d1857ca6892ea10 Mon Sep 17 00:00:00 2001 From: "Richard (Rick) Zamora" Date: Fri, 25 Mar 2022 10:48:26 -0500 Subject: [PATCH 01/44] Add basic predicate-pushdown optimization (#433) * basic predicate-pushdown support * remove explict Dispatch class * use _Frame.fillna * cleanup comments * test coverage * improve test coverage * add xfail test for dt accessor in predicate and fix test_show.py * fix some naming issues * add config and use assert_eq * add logging events when predicate-pushdown bails * move bail logic earlier in function * address easier code review comments * typo fix * fix creation_info access bug * convert any expression to DNF * csv test coverage * include IN coverage * improve test rigor * address code review * skip parquet tests when deps are not installed * fix bug * add pyarrow dep to cluster workers * roll back test skipping changes Co-authored-by: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> --- .github/docker-compose.yaml | 2 + dask_sql/physical/rel/logical/filter.py | 8 +- dask_sql/physical/utils/filter.py | 368 ++++++++++++++++++++++++ dask_sql/sql-schema.yaml | 5 + dask_sql/sql.yaml | 2 + tests/integration/fixtures.py | 32 ++- tests/integration/test_filter.py | 95 ++++++ tests/integration/test_show.py | 2 + 8 files changed, 512 insertions(+), 2 deletions(-) create mode 100644 dask_sql/physical/utils/filter.py diff --git a/.github/docker-compose.yaml b/.github/docker-compose.yaml index cfb7eb43f..56ec50b47 100644 --- a/.github/docker-compose.yaml +++ b/.github/docker-compose.yaml @@ -11,5 +11,7 @@ services: container_name: dask-worker image: daskdev/dask:latest command: dask-worker dask-scheduler:8786 + environment: + EXTRA_CONDA_PACKAGES: "pyarrow>1.0.0" # required for parquet IO volumes: - /tmp:/tmp diff --git a/dask_sql/physical/rel/logical/filter.py b/dask_sql/physical/rel/logical/filter.py index 87c99e3e0..6e7078efd 100644 --- a/dask_sql/physical/rel/logical/filter.py +++ b/dask_sql/physical/rel/logical/filter.py @@ -1,12 +1,14 @@ import logging from typing import TYPE_CHECKING, Union +import dask.config as dask_config import dask.dataframe as dd import numpy as np from dask_sql.datacontainer import DataContainer from dask_sql.physical.rel.base import BaseRelPlugin from dask_sql.physical.rex import RexConverter +from dask_sql.physical.utils.filter import attempt_predicate_pushdown if TYPE_CHECKING: import dask_sql @@ -31,7 +33,11 @@ def filter_or_scalar(df: dd.DataFrame, filter_condition: Union[np.bool_, dd.Seri # In SQL, a NULL in a boolean is False on filtering filter_condition = filter_condition.fillna(False) - return df[filter_condition] + out = df[filter_condition] + if dask_config.get("sql.predicate_pushdown"): + return attempt_predicate_pushdown(out) + else: + return out class DaskFilterPlugin(BaseRelPlugin): diff --git a/dask_sql/physical/utils/filter.py b/dask_sql/physical/utils/filter.py new file mode 100644 index 000000000..67e4026f5 --- /dev/null +++ b/dask_sql/physical/utils/filter.py @@ -0,0 +1,368 @@ +import itertools +import logging +import operator + +import dask.dataframe as dd +import numpy as np +from dask.blockwise import Blockwise +from dask.highlevelgraph import HighLevelGraph +from dask.layers import DataFrameIOLayer +from dask.utils import M, apply, is_arraylike + +logger = logging.getLogger(__name__) + + +def attempt_predicate_pushdown(ddf: dd.DataFrame) -> dd.DataFrame: + """Use graph information to update IO-level filters + + The original `ddf` will be returned if/when the + predicate-pushdown optimization fails. + + This is a special optimization that must be called + eagerly on a DataFrame collection when filters are + applied. The "eager" requirement for this optimization + is due to the fact that `npartitions` and `divisions` + may change when this optimization is applied (invalidating + npartition/divisions-specific logic in following Layers). + """ + + # Check that we have a supported `ddf` object + if not isinstance(ddf, dd.DataFrame): + raise ValueError( + f"Predicate pushdown optimization skipped. Type {type(ddf)} " + f"does not support predicate pushdown." + ) + elif not isinstance(ddf.dask, HighLevelGraph): + logger.warning( + f"Predicate pushdown optimization skipped. Graph must be " + f"a HighLevelGraph object (got {type(ddf.dask)})." + ) + return ddf + + # We were able to extract a DNF filter expression. + # Check that we have a single IO layer with `filters` support + io_layer = [] + for k, v in ddf.dask.layers.items(): + if isinstance(v, DataFrameIOLayer): + io_layer.append(k) + creation_info = ( + (v.creation_info or {}) if hasattr(v, "creation_info") else {} + ) + if ( + "filters" not in creation_info.get("kwargs", {}) + or creation_info["kwargs"]["filters"] is not None + ): + # No filters support, or filters is already set + return ddf + if len(io_layer) != 1: + # Not a single IO layer + return ddf + io_layer = io_layer.pop() + + # Start by converting the HLG to a `RegenerableGraph`. + # Succeeding here means that all layers in the graph + # are regenerable. + try: + dsk = RegenerableGraph.from_hlg(ddf.dask) + except (ValueError, TypeError): + logger.warning( + "Predicate pushdown optimization skipped. One or more " + "layers in the HighLevelGraph was not 'regenerable'." + ) + return ddf + + # Extract a DNF-formatted filter expression + name = ddf._name + try: + filters = dsk.layers[name]._dnf_filter_expression(dsk) + if not isinstance(filters, frozenset): + # No filters encountered + return ddf + filters = filters.to_list_tuple() + except ValueError: + # DNF dispatching failed for 1+ layers + logger.warning( + "Predicate pushdown optimization skipped. One or more " + "layers has an unknown filter expression." + ) + return ddf + + # Regenerate collection with filtered IO layer + try: + return dsk.layers[name]._regenerate_collection( + dsk, new_kwargs={io_layer: {"filters": filters}}, + ) + except ValueError as err: + # Most-likely failed to apply filters in read_parquet. + # We can just bail on predicate pushdown, but we also + # raise a warning to encourage the user to file an issue. + logger.warning( + f"Predicate pushdown failed to apply filters: {filters}. " + f"Please open a bug report at " + f"https://github.com/dask-contrib/dask-sql/issues/new/choose " + f"and include the following error message: {err}" + ) + + return ddf + + +class Or(frozenset): + """Helper class for 'OR' expressions""" + + def to_list_tuple(self): + # NDF "or" is List[List[Tuple]] + def _maybe_list(val): + if isinstance(val, tuple) and val and isinstance(val[0], (tuple, list)): + return list(val) + return [val] + + return [ + _maybe_list(val.to_list_tuple()) + if hasattr(val, "to_list_tuple") + else _maybe_list(val) + for val in self + ] + + +class And(frozenset): + """Helper class for 'AND' expressions""" + + def to_list_tuple(self): + # NDF "and" is List[Tuple] + return tuple( + val.to_list_tuple() if hasattr(val, "to_list_tuple") else val + for val in self + ) + + +def to_dnf(expr): + """Normalize a boolean filter expression to disjunctive normal form (DNF)""" + + # Credit: https://stackoverflow.com/a/58372345 + if not isinstance(expr, (Or, And)): + result = Or((And((expr,)),)) + elif isinstance(expr, Or): + result = Or(se for e in expr for se in to_dnf(e)) + elif isinstance(expr, And): + total = [] + for c in itertools.product(*[to_dnf(e) for e in expr]): + total.append(And(se for e in c for se in e)) + result = Or(total) + return result + + +# Define all supported comparison functions +# (and their mapping to a string expression) +_comparison_symbols = { + operator.eq: "==", + operator.ne: "!=", + operator.lt: "<", + operator.le: "<=", + operator.gt: ">", + operator.ge: ">=", + np.greater: ">", + np.greater_equal: ">=", + np.less: "<", + np.less_equal: "<=", + np.equal: "==", + np.not_equal: "!=", +} + +# Define set of all "regenerable" operations. +# Predicate pushdown is supported for graphs +# comprised of `Blockwise` layers based on these +# operations +_regenerable_ops = set(_comparison_symbols.keys()) | { + operator.and_, + operator.or_, + operator.getitem, + M.fillna, +} + +# Specify functions that must be generated with +# a different API at the dataframe-collection level +_special_op_mappings = {M.fillna: dd._Frame.fillna} + + +class RegenerableLayer: + """Regenerable Layer + + Wraps ``dask.highlevelgraph.Blockwise`` to ensure that a + ``creation_info`` attribute is defined. This class + also defines the necessary methods for recursive + layer regeneration and filter-expression generation. + """ + + def __init__(self, layer, creation_info): + self.layer = layer # Original Blockwise layer reference + self.creation_info = creation_info + + def _regenerate_collection( + self, dsk, new_kwargs: dict = None, _regen_cache: dict = None, + ): + """Regenerate a Dask collection for this layer using the + provided inputs and key-word arguments + """ + + # Return regenerated layer if the work was + # already done + _regen_cache = _regen_cache or {} + if self.layer.output in _regen_cache: + return _regen_cache[self.layer.output] + + # Recursively generate necessary inputs to + # this layer to generate the collection + inputs = [] + for key, ind in self.layer.indices: + if ind is None: + if isinstance(key, (str, tuple)) and key in dsk.layers: + continue + inputs.append(key) + elif key in self.layer.io_deps: + continue + else: + inputs.append( + dsk.layers[key]._regenerate_collection( + dsk, new_kwargs=new_kwargs, _regen_cache=_regen_cache, + ) + ) + + # Extract the callable func and key-word args. + # Then return a regenerated collection + func = self.creation_info.get("func", None) + if func is None: + raise ValueError( + "`_regenerate_collection` failed. " + "Not all HLG layers are regenerable." + ) + regen_args = self.creation_info.get("args", []) + regen_kwargs = self.creation_info.get("kwargs", {}).copy() + regen_kwargs = {k: v for k, v in self.creation_info.get("kwargs", {}).items()} + regen_kwargs.update((new_kwargs or {}).get(self.layer.output, {})) + result = func(*inputs, *regen_args, **regen_kwargs) + _regen_cache[self.layer.output] = result + return result + + def _dnf_filter_expression(self, dsk): + """Return a DNF-formatted filter expression for the + graph terminating at this layer + """ + op = self.creation_info["func"] + if op in _comparison_symbols.keys(): + func = _blockwise_comparison_dnf + elif op in (operator.and_, operator.or_): + func = _blockwise_logical_dnf + elif op == operator.getitem: + func = _blockwise_getitem_dnf + elif op == dd._Frame.fillna: + func = _blockwise_fillna_dnf + else: + raise ValueError(f"No DNF expression for {op}") + + return func(op, self.layer.indices, dsk) + + +class RegenerableGraph: + """Regenerable Graph + + This class is similar to ``dask.highlevelgraph.HighLevelGraph``. + However, all layers in a ``RegenerableGraph`` graph must be + ``RegenerableLayer`` objects (which wrap ``Blockwise`` layers). + """ + + def __init__(self, layers: dict): + self.layers = layers + + @classmethod + def from_hlg(cls, hlg: HighLevelGraph): + """Construct a ``RegenerableGraph`` from a ``HighLevelGraph``""" + + if not isinstance(hlg, HighLevelGraph): + raise TypeError(f"Expected HighLevelGraph, got {type(hlg)}") + + _layers = {} + for key, layer in hlg.layers.items(): + regenerable_layer = None + if isinstance(layer, DataFrameIOLayer): + regenerable_layer = RegenerableLayer(layer, layer.creation_info or {}) + elif isinstance(layer, Blockwise): + tasks = list(layer.dsk.values()) + if len(tasks) == 1 and tasks[0]: + kwargs = {} + if tasks[0][0] == apply: + op = tasks[0][1] + options = tasks[0][3] + if isinstance(options, dict): + kwargs = options + elif ( + isinstance(options, tuple) + and options + and callable(options[0]) + ): + kwargs = options[0](*options[1:]) + else: + op = tasks[0][0] + if op in _regenerable_ops: + regenerable_layer = RegenerableLayer( + layer, + { + "func": _special_op_mappings.get(op, op), + "kwargs": kwargs, + }, + ) + + if regenerable_layer is None: + raise ValueError(f"Graph contains non-regenerable layer: {layer}") + + _layers[key] = regenerable_layer + + return RegenerableGraph(_layers) + + +def _get_blockwise_input(input_index, indices: list, dsk: RegenerableGraph): + # Simple utility to get the required input expressions + # for a Blockwise layer (using indices) + key = indices[input_index][0] + if indices[input_index][1] is None: + return key + return dsk.layers[key]._dnf_filter_expression(dsk) + + +def _blockwise_comparison_dnf(op, indices: list, dsk: RegenerableGraph): + # Return DNF expression pattern for a simple comparison + left = _get_blockwise_input(0, indices, dsk) + right = _get_blockwise_input(1, indices, dsk) + + def _inv(symbol: str): + return {">": "<", "<": ">", ">=": "<=", "<=": ">=",}.get(symbol, symbol) + + if is_arraylike(left) and hasattr(left, "item") and left.size == 1: + left = left.item() + # Need inverse comparison in read_parquet + return (right, _inv(_comparison_symbols[op]), left) + if is_arraylike(right) and hasattr(right, "item") and right.size == 1: + right = right.item() + return to_dnf((left, _comparison_symbols[op], right)) + + +def _blockwise_logical_dnf(op, indices: list, dsk: RegenerableGraph): + # Return DNF expression pattern for logical "and" or "or" + left = _get_blockwise_input(0, indices, dsk) + right = _get_blockwise_input(1, indices, dsk) + if op == operator.or_: + return to_dnf(Or([left, right])) + elif op == operator.and_: + return to_dnf(And([left, right])) + else: + raise ValueError + + +def _blockwise_getitem_dnf(op, indices: list, dsk: RegenerableGraph): + # Return dnf of key (selected by getitem) + key = _get_blockwise_input(1, indices, dsk) + return key + + +def _blockwise_fillna_dnf(op, indices: list, dsk: RegenerableGraph): + # Return dnf of input collection + return _get_blockwise_input(0, indices, dsk) diff --git a/dask_sql/sql-schema.yaml b/dask_sql/sql-schema.yaml index 06c766854..f65e4d344 100644 --- a/dask_sql/sql-schema.yaml +++ b/dask_sql/sql-schema.yaml @@ -26,3 +26,8 @@ properties: type: boolean description: | Whether sql identifiers are considered case sensitive while parsing. + + predicate_pushdown: + type: bool + description: | + Whether to try pushing down filter predicates into IO (when possible). diff --git a/dask_sql/sql.yaml b/dask_sql/sql.yaml index 1976e72c3..72f28c271 100644 --- a/dask_sql/sql.yaml +++ b/dask_sql/sql.yaml @@ -5,3 +5,5 @@ sql: identifier: case_sensitive: True + + predicate_pushdown: True diff --git a/tests/integration/fixtures.py b/tests/integration/fixtures.py index 056c681e0..75b98a9f7 100644 --- a/tests/integration/fixtures.py +++ b/tests/integration/fixtures.py @@ -114,6 +114,30 @@ def datetime_table(): ) +@pytest.fixture() +def parquet_ddf(tmpdir): + + # Write simple parquet dataset + df = pd.DataFrame( + { + "a": [1, 2, 3] * 5, + "b": range(15), + "c": ["A"] * 15, + "d": [ + pd.Timestamp("2013-08-01 23:00:00"), + pd.Timestamp("2014-09-01 23:00:00"), + pd.Timestamp("2015-10-01 23:00:00"), + ] + * 5, + "index": range(15), + }, + ) + dd.from_pandas(df, npartitions=3).to_parquet(os.path.join(tmpdir, "parquet")) + + # Read back with dask and apply WHERE query + return dd.read_parquet(os.path.join(tmpdir, "parquet"), index="index") + + @pytest.fixture() def gpu_user_table_1(user_table_1): return cudf.from_pandas(user_table_1) if cudf else None @@ -151,6 +175,7 @@ def c( user_table_nan, string_table, datetime_table, + parquet_ddf, gpu_user_table_1, gpu_df, gpu_long_table, @@ -168,6 +193,7 @@ def c( "user_table_nan": user_table_nan, "string_table": string_table, "datetime_table": datetime_table, + "parquet_ddf": parquet_ddf, "gpu_user_table_1": gpu_user_table_1, "gpu_df": gpu_df, "gpu_long_table": gpu_long_table, @@ -182,7 +208,11 @@ def c( for df_name, df in dfs.items(): if df is None: continue - dask_df = dd.from_pandas(df, npartitions=3) + if hasattr(df, "npartitions"): + # df is already a dask collection + dask_df = df + else: + dask_df = dd.from_pandas(df, npartitions=3) c.create_table(df_name, dask_df) yield c diff --git a/tests/integration/test_filter.py b/tests/integration/test_filter.py index ad98d4416..345b9d9e1 100644 --- a/tests/integration/test_filter.py +++ b/tests/integration/test_filter.py @@ -1,6 +1,7 @@ import dask.dataframe as dd import pandas as pd import pytest +from dask.utils_test import hlg_layer from pandas.testing import assert_frame_equal from dask_sql._compat import INT_NAN_IMPLEMENTED @@ -122,3 +123,97 @@ def test_filter_year(c): expected_df = df[df["year"] < 2016] assert_frame_equal(expected_df, actual_df) + + +@pytest.mark.parametrize( + "query,df_func,filters", + [ + ( + "SELECT * FROM parquet_ddf WHERE b < 10", + lambda x: x[x["b"] < 10], + [[("b", "<", 10)]], + ), + ( + "SELECT * FROM parquet_ddf WHERE a < 3 AND (b > 1 AND b < 5)", + lambda x: x[(x["a"] < 3) & ((x["b"] > 1) & (x["b"] < 5))], + [[("a", "<", 3), ("b", ">", 1), ("b", "<", 5)]], + ), + ( + "SELECT * FROM parquet_ddf WHERE (b > 5 AND b < 10) OR a = 1", + lambda x: x[((x["b"] > 5) & (x["b"] < 10)) | (x["a"] == 1)], + [[("a", "==", 1)], [("b", "<", 10), ("b", ">", 5)]], + ), + ( + "SELECT * FROM parquet_ddf WHERE b IN (1, 6)", + lambda x: x[(x["b"] == 1) | (x["b"] == 6)], + [[("b", "<=", 1), ("b", ">=", 1)], [("b", "<=", 6), ("b", ">=", 6)]], + ), + ( + "SELECT a FROM parquet_ddf WHERE (b > 5 AND b < 10) OR a = 1", + lambda x: x[((x["b"] > 5) & (x["b"] < 10)) | (x["a"] == 1)][["a"]], + [[("a", "==", 1)], [("b", "<", 10), ("b", ">", 5)]], + ), + ( + # Original filters NOT in disjunctive normal form + "SELECT a FROM parquet_ddf WHERE (parquet_ddf.b > 3 AND parquet_ddf.b < 10 OR parquet_ddf.a = 1) AND (parquet_ddf.c = 'A')", + lambda x: x[ + ((x["b"] > 3) & (x["b"] < 10) | (x["a"] == 1)) & (x["c"] == "A") + ][["a"]], + [ + [("c", "==", "A"), ("b", ">", 3), ("b", "<", 10)], + [("a", "==", 1), ("c", "==", "A")], + ], + ), + ( + # The predicate-pushdown optimization will be skipped here, + # because datetime accessors are not supported. However, + # the query should still succeed. + "SELECT * FROM parquet_ddf WHERE year(d) < 2015", + lambda x: x[x["d"].dt.year < 2015], + None, + ), + ], +) +def test_predicate_pushdown(c, parquet_ddf, query, df_func, filters): + + # Check for predicate pushdown. + # We can use the `hlg_layer` utility to make sure the + # `filters` field has been populated in `creation_info` + return_df = c.sql(query) + expect_filters = filters + got_filters = hlg_layer(return_df.dask, "read-parquet").creation_info["kwargs"][ + "filters" + ] + if expect_filters: + got_filters = frozenset(frozenset(v) for v in got_filters) + expect_filters = frozenset(frozenset(v) for v in filters) + assert got_filters == expect_filters + + # Check computed result is correct + df = parquet_ddf.compute() + expected_df = df_func(df) + dd.assert_eq(return_df, expected_df) + + +def test_filtered_csv(tmpdir, c): + # Predicate pushdown is NOT supported for CSV data. + # This test just checks that the "attempted" + # predicate-pushdown logic does not lead to + # any unexpected errors + + # Write simple csv dataset + df = pd.DataFrame({"a": [1, 2, 3] * 5, "b": range(15), "c": ["A"] * 15,},) + dd.from_pandas(df, npartitions=3).to_csv(tmpdir + "/*.csv", index=False) + + # Read back with dask and apply WHERE query + csv_ddf = dd.read_csv(tmpdir + "/*.csv") + try: + c.create_table("my_csv_table", csv_ddf) + return_df = c.sql("SELECT * FROM my_csv_table WHERE b < 10") + finally: + c.drop_table("my_csv_table") + + # Check computed result is correct + df = csv_ddf.compute() + expected_df = df[df["b"] < 10] + dd.assert_eq(return_df, expected_df) diff --git a/tests/integration/test_show.py b/tests/integration/test_show.py index a04129489..41e315a95 100644 --- a/tests/integration/test_show.py +++ b/tests/integration/test_show.py @@ -43,6 +43,7 @@ def test_tables(c): "user_table_nan", "string_table", "datetime_table", + "parquet_ddf", ] if cudf is None else [ @@ -56,6 +57,7 @@ def test_tables(c): "user_table_nan", "string_table", "datetime_table", + "parquet_ddf", "gpu_user_table_1", "gpu_df", "gpu_long_table", From 09c7bdff29c6c0b082d679a3e18b1cc17d4ab9fb Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Fri, 25 Mar 2022 16:18:56 -0400 Subject: [PATCH 02/44] Add workflow to keep datafusion dev branch up to date (#440) --- .github/workflows/datafusion-sync.yml | 30 +++++++++++++++++++++++++++ 1 file changed, 30 insertions(+) create mode 100644 .github/workflows/datafusion-sync.yml diff --git a/.github/workflows/datafusion-sync.yml b/.github/workflows/datafusion-sync.yml new file mode 100644 index 000000000..fd544eeae --- /dev/null +++ b/.github/workflows/datafusion-sync.yml @@ -0,0 +1,30 @@ +name: Keep datafusion branch up to date +on: + push: + branches: + - main + +# When this workflow is queued, automatically cancel any previous running +# or pending jobs +concurrency: + group: datafusion-sync + cancel-in-progress: true + +jobs: + sync-branches: + runs-on: ubuntu-latest + if: github.repository == 'dask-contrib/dask-sql' + steps: + - name: Checkout + uses: actions/checkout@v2 + - name: Set up Node + uses: actions/setup-node@v2 + with: + node-version: 12 + - name: Opening pull request + id: pull + uses: tretuna/sync-branches@1.4.0 + with: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + FROM_BRANCH: main + TO_BRANCH: datafusion-sql-planner From 1b0b6f7edef4d73ba1aa7f790488c2ad3ecd725c Mon Sep 17 00:00:00 2001 From: "github-actions[bot]" <41898282+github-actions[bot]@users.noreply.github.com> Date: Fri, 1 Apr 2022 12:50:38 -0400 Subject: [PATCH 03/44] Update gpuCI `RAPIDS_VER` to `22.06` (#434) Co-authored-by: github-actions[bot] <41898282+github-actions[bot]@users.noreply.github.com> --- continuous_integration/gpuci/axis.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/continuous_integration/gpuci/axis.yaml b/continuous_integration/gpuci/axis.yaml index 922eee23c..41ddb56ec 100644 --- a/continuous_integration/gpuci/axis.yaml +++ b/continuous_integration/gpuci/axis.yaml @@ -8,6 +8,6 @@ LINUX_VER: - ubuntu18.04 RAPIDS_VER: -- "22.04" +- "22.06" excludes: From a05138da1b22fe0026f39b40f31a9dc789381a37 Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Mon, 4 Apr 2022 10:30:41 -0400 Subject: [PATCH 04/44] Bump black to 22.3.0 (#443) --- .pre-commit-config.yaml | 2 +- .../environment-3.10-jdk11-dev.yaml | 2 +- .../environment-3.10-jdk8-dev.yaml | 2 +- .../environment-3.8-jdk11-dev.yaml | 2 +- .../environment-3.8-jdk8-dev.yaml | 2 +- .../environment-3.9-jdk11-dev.yaml | 2 +- .../environment-3.9-jdk8-dev.yaml | 2 +- dask_sql/cmd.py | 9 ++++--- dask_sql/context.py | 3 ++- dask_sql/input_utils/convert.py | 6 ++++- dask_sql/java.py | 5 +++- dask_sql/physical/rel/custom/analyze.py | 5 +++- dask_sql/physical/rel/logical/aggregate.py | 5 +++- dask_sql/physical/rel/logical/join.py | 6 ++++- dask_sql/physical/rel/logical/window.py | 3 ++- dask_sql/physical/rex/core/call.py | 25 +++++++++++++---- dask_sql/physical/utils/filter.py | 19 ++++++++++--- dask_sql/server/app.py | 4 ++- docker/conda.txt | 2 +- setup.py | 8 ++++-- tests/integration/fixtures.py | 5 +++- tests/integration/test_cmd.py | 3 ++- tests/integration/test_create.py | 6 ++++- tests/integration/test_filter.py | 27 ++++++++++++++----- tests/integration/test_function.py | 10 +++---- tests/integration/test_groupby.py | 11 ++++++-- tests/integration/test_join.py | 19 +++++++++---- tests/integration/test_rex.py | 8 ++++-- tests/integration/test_schema.py | 2 +- tests/integration/test_select.py | 16 ++++++++--- tests/integration/test_show.py | 5 +++- tests/integration/test_sort.py | 16 +++++++++-- tests/integration/test_union.py | 3 ++- tests/unit/test_call.py | 8 ++++-- tests/unit/test_context.py | 24 ++++++++++++++--- tests/unit/test_utils.py | 8 ++++-- 36 files changed, 217 insertions(+), 68 deletions(-) diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index e50a5f1d5..50af0bbb4 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -1,6 +1,6 @@ repos: - repo: https://github.com/psf/black - rev: 19.10b0 + rev: 22.3.0 hooks: - id: black language_version: python3 diff --git a/continuous_integration/environment-3.10-jdk11-dev.yaml b/continuous_integration/environment-3.10-jdk11-dev.yaml index e58e00c87..eae7fc83d 100644 --- a/continuous_integration/environment-3.10-jdk11-dev.yaml +++ b/continuous_integration/environment-3.10-jdk11-dev.yaml @@ -5,7 +5,7 @@ channels: dependencies: - adagio>=0.2.3 - antlr4-python3-runtime>=4.9.2 -- black=19.10b0 +- black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 - dask>=2021.11.1 diff --git a/continuous_integration/environment-3.10-jdk8-dev.yaml b/continuous_integration/environment-3.10-jdk8-dev.yaml index 4ad533e94..59183a513 100644 --- a/continuous_integration/environment-3.10-jdk8-dev.yaml +++ b/continuous_integration/environment-3.10-jdk8-dev.yaml @@ -5,7 +5,7 @@ channels: dependencies: - adagio>=0.2.3 - antlr4-python3-runtime>=4.9.2 -- black=19.10b0 +- black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 - dask>=2021.11.1 diff --git a/continuous_integration/environment-3.8-jdk11-dev.yaml b/continuous_integration/environment-3.8-jdk11-dev.yaml index ddf84ab5a..1a00258f7 100644 --- a/continuous_integration/environment-3.8-jdk11-dev.yaml +++ b/continuous_integration/environment-3.8-jdk11-dev.yaml @@ -5,7 +5,7 @@ channels: dependencies: - adagio>=0.2.3 - antlr4-python3-runtime>=4.9.2 -- black=19.10b0 +- black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 - dask>=2021.11.1 diff --git a/continuous_integration/environment-3.8-jdk8-dev.yaml b/continuous_integration/environment-3.8-jdk8-dev.yaml index bc6980584..9aaf4cd95 100644 --- a/continuous_integration/environment-3.8-jdk8-dev.yaml +++ b/continuous_integration/environment-3.8-jdk8-dev.yaml @@ -5,7 +5,7 @@ channels: dependencies: - adagio>=0.2.3 - antlr4-python3-runtime>=4.9.2 -- black=19.10b0 +- black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 - dask>=2021.11.1 diff --git a/continuous_integration/environment-3.9-jdk11-dev.yaml b/continuous_integration/environment-3.9-jdk11-dev.yaml index 7f7b19be2..df68fedd4 100644 --- a/continuous_integration/environment-3.9-jdk11-dev.yaml +++ b/continuous_integration/environment-3.9-jdk11-dev.yaml @@ -5,7 +5,7 @@ channels: dependencies: - adagio>=0.2.3 - antlr4-python3-runtime>=4.9.2 -- black=19.10b0 +- black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 - dask>=2021.11.1 diff --git a/continuous_integration/environment-3.9-jdk8-dev.yaml b/continuous_integration/environment-3.9-jdk8-dev.yaml index a7f35a132..f27acff9c 100644 --- a/continuous_integration/environment-3.9-jdk8-dev.yaml +++ b/continuous_integration/environment-3.9-jdk8-dev.yaml @@ -5,7 +5,7 @@ channels: dependencies: - adagio>=0.2.3 - antlr4-python3-runtime>=4.9.2 -- black=19.10b0 +- black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 - dask>=2021.11.1 diff --git a/dask_sql/cmd.py b/dask_sql/cmd.py index a7ed0c6e2..7bd5894d1 100644 --- a/dask_sql/cmd.py +++ b/dask_sql/cmd.py @@ -78,8 +78,8 @@ def _parse_meta_command(sql): def _meta_commands(sql: str, context: Context, client: Client) -> Union[bool, Client]: """ - parses metacommands and prints their result - returns True if meta commands detected + parses metacommands and prints their result + returns True if meta commands detected """ cmd, schema_name = _parse_meta_command(sql) available_commands = [ @@ -147,7 +147,10 @@ def _meta_commands(sql: str, context: Context, client: Client) -> Union[bool, Cl def cmd_loop( - context: Context = None, client: Client = None, startup=False, log_level=None, + context: Context = None, + client: Client = None, + startup=False, + log_level=None, ): # pragma: no cover """ Run a REPL for answering SQL queries using ``dask-sql``. diff --git a/dask_sql/context.py b/dask_sql/context.py index 008ce76c1..98cc46e21 100644 --- a/dask_sql/context.py +++ b/dask_sql/context.py @@ -908,7 +908,8 @@ def _register_callable( if replace: schema.function_lists = list( filter( - lambda f: f.name.lower() != lower_name, schema.function_lists, + lambda f: f.name.lower() != lower_name, + schema.function_lists, ) ) del schema.functions[lower_name] diff --git a/dask_sql/input_utils/convert.py b/dask_sql/input_utils/convert.py index e43df2334..d730926ee 100644 --- a/dask_sql/input_utils/convert.py +++ b/dask_sql/input_utils/convert.py @@ -55,7 +55,11 @@ def to_dc( maybe persist them to cluster memory before. """ filled_get_dask_dataframe = lambda *args: cls._get_dask_dataframe( - *args, table_name=table_name, format=format, gpu=gpu, **kwargs, + *args, + table_name=table_name, + format=format, + gpu=gpu, + **kwargs, ) if isinstance(input_item, list): diff --git a/dask_sql/java.py b/dask_sql/java.py index adbbc50a9..27d340221 100644 --- a/dask_sql/java.py +++ b/dask_sql/java.py @@ -86,7 +86,10 @@ def _set_or_check_java_home(): logger.debug(f"Starting JVM from path {jvmpath}...") jpype.startJVM( - *jvmArgs, ignoreUnrecognized=True, convertStrings=False, jvmpath=jvmpath, + *jvmArgs, + ignoreUnrecognized=True, + convertStrings=False, + jvmpath=jvmpath, ) logger.debug("...having started JVM") diff --git a/dask_sql/physical/rel/custom/analyze.py b/dask_sql/physical/rel/custom/analyze.py index 6bd7a1bfa..860e22c2e 100644 --- a/dask_sql/physical/rel/custom/analyze.py +++ b/dask_sql/physical/rel/custom/analyze.py @@ -61,7 +61,10 @@ def convert( ) ) statistics = statistics.append( - pd.Series({col: col for col in columns}, name="col_name",) + pd.Series( + {col: col for col in columns}, + name="col_name", + ) ) cc = ColumnContainer(statistics.columns) diff --git a/dask_sql/physical/rel/logical/aggregate.py b/dask_sql/physical/rel/logical/aggregate.py index c8cfab62c..cae4e42ed 100644 --- a/dask_sql/physical/rel/logical/aggregate.py +++ b/dask_sql/physical/rel/logical/aggregate.py @@ -179,7 +179,10 @@ def convert( # Do all aggregates df_result, output_column_order = self._do_aggregations( - rel, dc, group_columns, context, + rel, + dc, + group_columns, + context, ) # SQL does not care about the index, but we do not want to have any multiindices diff --git a/dask_sql/physical/rel/logical/join.py b/dask_sql/physical/rel/logical/join.py index 1f2f02ed6..925396c91 100644 --- a/dask_sql/physical/rel/logical/join.py +++ b/dask_sql/physical/rel/logical/join.py @@ -100,7 +100,11 @@ def convert( # The resulting dataframe will contain all (renamed) columns from the lhs and rhs # plus the added columns df = self._join_on_columns( - df_lhs_renamed, df_rhs_renamed, lhs_on, rhs_on, join_type, + df_lhs_renamed, + df_rhs_renamed, + lhs_on, + rhs_on, + join_type, ) else: # 5. We are in the complex join case diff --git a/dask_sql/physical/rel/logical/window.py b/dask_sql/physical/rel/logical/window.py index adf3e6da3..e8541a41c 100644 --- a/dask_sql/physical/rel/logical/window.py +++ b/dask_sql/physical/rel/logical/window.py @@ -176,7 +176,8 @@ def map_on_each_group( upper_bound.is_current_row or upper_bound.offset == 0 ): windowed_group = partitioned_group.rolling( - window=lower_bound.offset + 1, min_periods=0, + window=lower_bound.offset + 1, + min_periods=0, ) else: lower_offset = lower_bound.offset if not lower_bound.is_current_row else 0 diff --git a/dask_sql/physical/rex/core/call.py b/dask_sql/physical/rex/core/call.py index b17ad38f7..55580e741 100644 --- a/dask_sql/physical/rex/core/call.py +++ b/dask_sql/physical/rex/core/call.py @@ -246,7 +246,10 @@ class IsFalseOperation(Operation): def __init__(self): super().__init__(self.false_) - def false_(self, df: SeriesOrScalar,) -> SeriesOrScalar: + def false_( + self, + df: SeriesOrScalar, + ) -> SeriesOrScalar: """ Returns true where `df` is false (where `df` can also be just a scalar). Returns false on nan. @@ -263,7 +266,10 @@ class IsTrueOperation(Operation): def __init__(self): super().__init__(self.true_) - def true_(self, df: SeriesOrScalar,) -> SeriesOrScalar: + def true_( + self, + df: SeriesOrScalar, + ) -> SeriesOrScalar: """ Returns true where `df` is true (where `df` can also be just a scalar). Returns false on nan. @@ -280,7 +286,10 @@ class NotOperation(Operation): def __init__(self): super().__init__(self.not_) - def not_(self, df: SeriesOrScalar,) -> SeriesOrScalar: + def not_( + self, + df: SeriesOrScalar, + ) -> SeriesOrScalar: """ Returns not `df` (where `df` can also be just a scalar). """ @@ -296,7 +305,10 @@ class IsNullOperation(Operation): def __init__(self): super().__init__(self.null) - def null(self, df: SeriesOrScalar,) -> SeriesOrScalar: + def null( + self, + df: SeriesOrScalar, + ) -> SeriesOrScalar: """ Returns true where `df` is null (where `df` can also be just a scalar). """ @@ -328,7 +340,10 @@ def __init__(self): super().__init__(self.regex) def regex( - self, test: SeriesOrScalar, regex: str, escape: str = None, + self, + test: SeriesOrScalar, + regex: str, + escape: str = None, ) -> SeriesOrScalar: """ Returns true, if the string test matches the given regex diff --git a/dask_sql/physical/utils/filter.py b/dask_sql/physical/utils/filter.py index 67e4026f5..ddfacd6ab 100644 --- a/dask_sql/physical/utils/filter.py +++ b/dask_sql/physical/utils/filter.py @@ -90,7 +90,8 @@ def attempt_predicate_pushdown(ddf: dd.DataFrame) -> dd.DataFrame: # Regenerate collection with filtered IO layer try: return dsk.layers[name]._regenerate_collection( - dsk, new_kwargs={io_layer: {"filters": filters}}, + dsk, + new_kwargs={io_layer: {"filters": filters}}, ) except ValueError as err: # Most-likely failed to apply filters in read_parquet. @@ -198,7 +199,10 @@ def __init__(self, layer, creation_info): self.creation_info = creation_info def _regenerate_collection( - self, dsk, new_kwargs: dict = None, _regen_cache: dict = None, + self, + dsk, + new_kwargs: dict = None, + _regen_cache: dict = None, ): """Regenerate a Dask collection for this layer using the provided inputs and key-word arguments @@ -223,7 +227,9 @@ def _regenerate_collection( else: inputs.append( dsk.layers[key]._regenerate_collection( - dsk, new_kwargs=new_kwargs, _regen_cache=_regen_cache, + dsk, + new_kwargs=new_kwargs, + _regen_cache=_regen_cache, ) ) @@ -334,7 +340,12 @@ def _blockwise_comparison_dnf(op, indices: list, dsk: RegenerableGraph): right = _get_blockwise_input(1, indices, dsk) def _inv(symbol: str): - return {">": "<", "<": ">", ">=": "<=", "<=": ">=",}.get(symbol, symbol) + return { + ">": "<", + "<": ">", + ">=": "<=", + "<=": ">=", + }.get(symbol, symbol) if is_arraylike(left) and hasattr(left, "item") and left.size == 1: left = left.item() diff --git a/dask_sql/server/app.py b/dask_sql/server/app.py index 34217629e..634de3856 100644 --- a/dask_sql/server/app.py +++ b/dask_sql/server/app.py @@ -272,7 +272,9 @@ def main(): # pragma: no cover def _init_app( - app: FastAPI, context: Context = None, client: dask.distributed.Client = None, + app: FastAPI, + context: Context = None, + client: dask.distributed.Client = None, ): app.c = context or Context() app.future_list = {} diff --git a/docker/conda.txt b/docker/conda.txt index b008852cd..d833ac877 100644 --- a/docker/conda.txt +++ b/docker/conda.txt @@ -20,5 +20,5 @@ dask-ml>=2022.1.22 scikit-learn>=0.24.2 intake>=0.6.0 pre-commit>=2.11.1 -black=19.10b0 +black=22.3.0 isort=5.7.0 diff --git a/setup.py b/setup.py index 2d04f3320..e63b9a3d5 100755 --- a/setup.py +++ b/setup.py @@ -109,7 +109,7 @@ def build(self): "scikit-learn>=0.24.2", "intake>=0.6.0", "pre-commit", - "black==19.10b0", + "black==22.3.0", "isort==5.7.0", ], "fugue": ["fugue[sql]>=0.5.3"], @@ -122,5 +122,9 @@ def build(self): }, zip_safe=False, cmdclass=cmdclass, - command_options={"build_sphinx": {"source_dir": ("setup.py", "docs"),}}, + command_options={ + "build_sphinx": { + "source_dir": ("setup.py", "docs"), + } + }, ) diff --git a/tests/integration/fixtures.py b/tests/integration/fixtures.py index 75b98a9f7..5be6a930b 100644 --- a/tests/integration/fixtures.py +++ b/tests/integration/fixtures.py @@ -57,7 +57,10 @@ def df_wide(): def df(): np.random.seed(42) return pd.DataFrame( - {"a": [1.0] * 100 + [2.0] * 200 + [3.0] * 400, "b": 10 * np.random.rand(700),} + { + "a": [1.0] * 100 + [2.0] * 200 + [3.0] * 400, + "b": 10 * np.random.rand(700), + } ) diff --git a/tests/integration/test_cmd.py b/tests/integration/test_cmd.py index dd8b05083..145b4962f 100644 --- a/tests/integration/test_cmd.py +++ b/tests/integration/test_cmd.py @@ -100,7 +100,8 @@ def test_meta_commands(c, client, capsys): assert "Schema not_exists not available\n" == captured.out with pytest.raises( - OSError, match="Timed out .* to tcp://localhost:8787 after 5 s", + OSError, + match="Timed out .* to tcp://localhost:8787 after 5 s", ): with dask_config.set({"distributed.comm.timeouts.connect": 5}): client = _meta_commands("\\dsc localhost:8787", context=c, client=client) diff --git a/tests/integration/test_create.py b/tests/integration/test_create.py index b6d513f4e..cac4a3099 100644 --- a/tests/integration/test_create.py +++ b/tests/integration/test_create.py @@ -35,7 +35,11 @@ def test_create_from_csv(c, df, temporary_data_file, gpu): @pytest.mark.parametrize( - "gpu", [False, pytest.param(True, marks=pytest.mark.gpu),], + "gpu", + [ + False, + pytest.param(True, marks=pytest.mark.gpu), + ], ) def test_cluster_memory(client, c, df, gpu): client.publish_dataset(df=dd.from_pandas(df, npartitions=1)) diff --git a/tests/integration/test_filter.py b/tests/integration/test_filter.py index 345b9d9e1..a5231a5e2 100644 --- a/tests/integration/test_filter.py +++ b/tests/integration/test_filter.py @@ -47,7 +47,8 @@ def test_filter_complicated(c, df): expected_df = df[((df["a"] < 3) & ((df["b"] > 1) & (df["b"] < 3)))] assert_frame_equal( - return_df, expected_df, + return_df, + expected_df, ) @@ -60,7 +61,8 @@ def test_filter_with_nan(c): else: expected_df = pd.DataFrame({"c": [3]}, dtype="float") assert_frame_equal( - return_df, expected_df, + return_df, + expected_df, ) @@ -69,13 +71,17 @@ def test_string_filter(c, string_table): return_df = return_df.compute() assert_frame_equal( - return_df, string_table.head(1), + return_df, + string_table.head(1), ) @pytest.mark.parametrize( "input_table", - ["datetime_table", pytest.param("gpu_datetime_table", marks=pytest.mark.gpu),], + [ + "datetime_table", + pytest.param("gpu_datetime_table", marks=pytest.mark.gpu), + ], ) def test_filter_cast_date(c, input_table, request): datetime_table = request.getfixturevalue(input_table) @@ -95,7 +101,10 @@ def test_filter_cast_date(c, input_table, request): @pytest.mark.parametrize( "input_table", - ["datetime_table", pytest.param("gpu_datetime_table", marks=pytest.mark.gpu),], + [ + "datetime_table", + pytest.param("gpu_datetime_table", marks=pytest.mark.gpu), + ], ) def test_filter_cast_timestamp(c, input_table, request): datetime_table = request.getfixturevalue(input_table) @@ -202,7 +211,13 @@ def test_filtered_csv(tmpdir, c): # any unexpected errors # Write simple csv dataset - df = pd.DataFrame({"a": [1, 2, 3] * 5, "b": range(15), "c": ["A"] * 15,},) + df = pd.DataFrame( + { + "a": [1, 2, 3] * 5, + "b": range(15), + "c": ["A"] * 15, + }, + ) dd.from_pandas(df, npartitions=3).to_csv(tmpdir + "/*.csv", index=False) # Read back with dask and apply WHERE query diff --git a/tests/integration/test_function.py b/tests/integration/test_function.py index a116e2459..c6342877c 100644 --- a/tests/integration/test_function.py +++ b/tests/integration/test_function.py @@ -9,7 +9,7 @@ def test_custom_function(c, df): def f(x): - return x ** 2 + return x**2 c.register_function(f, "f", [("x", np.float64)], np.float64) @@ -146,7 +146,7 @@ def f(row, k1, k2): def test_multiple_definitions(c, df_simple): def f(x): - return x ** 2 + return x**2 c.register_function(f, "f", [("x", np.float64)], np.float64) c.register_function(f, "f", [("x", np.int64)], np.int64) @@ -162,7 +162,7 @@ def f(x): assert_frame_equal(return_df.reset_index(drop=True), df_simple[["a", "b"]] ** 2) def f(x): - return x ** 3 + return x**3 c.register_function(f, "f", [("x", np.float64)], np.float64, replace=True) c.register_function(f, "f", [("x", np.int64)], np.int64) @@ -195,14 +195,14 @@ def test_aggregate_function(c): def test_reregistration(c): def f(x): - return x ** 2 + return x**2 # The same is fine c.register_function(f, "f", [("x", np.float64)], np.float64) c.register_function(f, "f", [("x", np.int64)], np.int64) def f(x): - return x ** 3 + return x**3 # A different not with pytest.raises(ValueError): diff --git a/tests/integration/test_groupby.py b/tests/integration/test_groupby.py index e6eba9060..0fef45679 100644 --- a/tests/integration/test_groupby.py +++ b/tests/integration/test_groupby.py @@ -93,7 +93,11 @@ def test_group_by_filtered2(c): df = df.compute() expected_df = pd.DataFrame( - {"user_id": [1, 2, 3], "S1": [np.NaN, 4.0, np.NaN], "S2": [3, 4, 3],}, + { + "user_id": [1, 2, 3], + "S1": [np.NaN, 4.0, np.NaN], + "S2": [3, 4, 3], + }, ) assert_frame_equal(df, expected_df) @@ -351,7 +355,10 @@ def test_stats_aggregation(c, timeseries_df): @pytest.mark.parametrize( "input_table", - ["user_table_1", pytest.param("gpu_user_table_1", marks=pytest.mark.gpu),], + [ + "user_table_1", + pytest.param("gpu_user_table_1", marks=pytest.mark.gpu), + ], ) @pytest.mark.parametrize("split_out", [None, 2, 4]) def test_groupby_split_out(c, input_table, split_out, request): diff --git a/tests/integration/test_join.py b/tests/integration/test_join.py index fa9d626f9..44cd1e070 100644 --- a/tests/integration/test_join.py +++ b/tests/integration/test_join.py @@ -17,7 +17,8 @@ def test_join(c): {"user_id": [1, 1, 2, 2], "b": [3, 3, 1, 3], "c": [1, 2, 3, 3]} ) assert_frame_equal( - df.sort_values(["user_id", "b", "c"]).reset_index(drop=True), expected_df, + df.sort_values(["user_id", "b", "c"]).reset_index(drop=True), + expected_df, ) @@ -31,7 +32,8 @@ def test_join_inner(c): {"user_id": [1, 1, 2, 2], "b": [3, 3, 1, 3], "c": [1, 2, 3, 3]} ) assert_frame_equal( - df.sort_values(["user_id", "b", "c"]).reset_index(drop=True), expected_df, + df.sort_values(["user_id", "b", "c"]).reset_index(drop=True), + expected_df, ) @@ -71,7 +73,8 @@ def test_join_left(c): } ) assert_frame_equal( - df.sort_values(["user_id", "b", "c"]).reset_index(drop=True), expected_df, + df.sort_values(["user_id", "b", "c"]).reset_index(drop=True), + expected_df, ) @@ -91,7 +94,8 @@ def test_join_right(c): } ) assert_frame_equal( - df.sort_values(["user_id", "b", "c"]).reset_index(drop=True), expected_df, + df.sort_values(["user_id", "b", "c"]).reset_index(drop=True), + expected_df, ) @@ -119,7 +123,12 @@ def test_join_complex(c): df = df.compute() df_expected = pd.DataFrame( - {"a": [1, 1, 2], "b": [1.1, 1.1, 2.2], "a0": [2, 3, 3], "b0": [2.2, 3.3, 3.3],} + { + "a": [1, 1, 2], + "b": [1.1, 1.1, 2.2], + "a0": [2, 3, 3], + "b0": [2.2, 3.3, 3.3], + } ) assert_frame_equal(df.sort_values(["a", "b0"]).reset_index(drop=True), df_expected) diff --git a/tests/integration/test_rex.py b/tests/integration/test_rex.py index 1b870699f..aa6096b9e 100644 --- a/tests/integration/test_rex.py +++ b/tests/integration/test_rex.py @@ -124,7 +124,10 @@ def test_random(c, df): @pytest.mark.parametrize( "input_table", - ["string_table", pytest.param("gpu_string_table", marks=pytest.mark.gpu),], + [ + "string_table", + pytest.param("gpu_string_table", marks=pytest.mark.gpu), + ], ) def test_not(c, input_table, request): string_table = request.getfixturevalue(input_table) @@ -505,7 +508,8 @@ def test_string_functions(c, gpu): ) assert_frame_equal( - df.head(1), expected_df, + df.head(1), + expected_df, ) diff --git a/tests/integration/test_schema.py b/tests/integration/test_schema.py index bf36e90dc..50cc90c3d 100644 --- a/tests/integration/test_schema.py +++ b/tests/integration/test_schema.py @@ -40,7 +40,7 @@ def test_function(c): c.sql("USE SCHEMA root") def f(x): - return x ** 2 + return x**2 c.register_function(f, "f", [("x", np.float64)], np.float64, schema_name="other") diff --git a/tests/integration/test_select.py b/tests/integration/test_select.py index f7c20df3d..c6ce6a5a7 100644 --- a/tests/integration/test_select.py +++ b/tests/integration/test_select.py @@ -58,7 +58,11 @@ def test_select_expr(c, df): result_df = result_df.compute() expected_df = pd.DataFrame( - {"a": df["a"] + 1, "bla": df["b"], '"df"."a" - 1': df["a"] - 1,} + { + "a": df["a"] + 1, + "bla": df["b"], + '"df"."a" - 1': df["a"] - 1, + } ) assert_frame_equal(result_df, expected_df) @@ -121,7 +125,10 @@ def test_timezones(c, datetime_table): @pytest.mark.parametrize( "input_table", - ["datetime_table", pytest.param("gpu_datetime_table", marks=pytest.mark.gpu),], + [ + "datetime_table", + pytest.param("gpu_datetime_table", marks=pytest.mark.gpu), + ], ) def test_date_casting(c, input_table, request): datetime_table = request.getfixturevalue(input_table) @@ -151,7 +158,10 @@ def test_date_casting(c, input_table, request): @pytest.mark.parametrize( "input_table", - ["datetime_table", pytest.param("gpu_datetime_table", marks=pytest.mark.gpu),], + [ + "datetime_table", + pytest.param("gpu_datetime_table", marks=pytest.mark.gpu), + ], ) def test_timestamp_casting(c, input_table, request): datetime_table = request.getfixturevalue(input_table) diff --git a/tests/integration/test_show.py b/tests/integration/test_show.py index 41e315a95..c9d217b66 100644 --- a/tests/integration/test_show.py +++ b/tests/integration/test_show.py @@ -78,7 +78,10 @@ def test_columns(c): expected_df = pd.DataFrame( { - "Column": ["user_id", "b",], + "Column": [ + "user_id", + "b", + ], "Type": ["bigint", "bigint"], "Extra": [""] * 2, "Comment": [""] * 2, diff --git a/tests/integration/test_sort.py b/tests/integration/test_sort.py index 5825e3567..9023da438 100644 --- a/tests/integration/test_sort.py +++ b/tests/integration/test_sort.py @@ -270,7 +270,12 @@ def test_sort_with_nan_many_partitions(gpu): xd = pd c = Context() - df = xd.DataFrame({"a": [float("nan"), 1] * 30, "b": [1, 2, 3] * 20,}) + df = xd.DataFrame( + { + "a": [float("nan"), 1] * 30, + "b": [1, 2, 3] * 20, + } + ) c.create_table("df", dd.from_pandas(df, npartitions=10)) df_result = ( @@ -294,7 +299,14 @@ def test_sort_with_nan_many_partitions(gpu): df_result = c.sql("SELECT * FROM df ORDER BY a").compute().reset_index(drop=True) - dd.assert_eq(df_result, xd.DataFrame({"a": [1] * 30 + [float("nan")] * 30,})) + dd.assert_eq( + df_result, + xd.DataFrame( + { + "a": [1] * 30 + [float("nan")] * 30, + } + ), + ) @pytest.mark.parametrize("gpu", [False, pytest.param(True, marks=pytest.mark.gpu)]) diff --git a/tests/integration/test_union.py b/tests/integration/test_union.py index 801102b95..e199b45d1 100644 --- a/tests/integration/test_union.py +++ b/tests/integration/test_union.py @@ -47,6 +47,7 @@ def test_union_mixed(c, df, long_table): long_table["II"] = long_table["I"] expected_df = pd.concat( - [df.rename(columns={"a": "I", "b": "II"}), long_table], ignore_index=True, + [df.rename(columns={"a": "I", "b": "II"}), long_table], + ignore_index=True, ) assert_frame_equal(result_df.reset_index(drop=True), expected_df) diff --git a/tests/unit/test_call.py b/tests/unit/test_call.py index 255c465bb..2af33551d 100644 --- a/tests/unit/test_call.py +++ b/tests/unit/test_call.py @@ -157,10 +157,14 @@ def test_simple_ops(): def test_math_operations(): assert_series_equal( - ops_mapping["abs"](-df1.a).compute(), pd.Series([1, 2, 3]), check_names=False, + ops_mapping["abs"](-df1.a).compute(), + pd.Series([1, 2, 3]), + check_names=False, ) assert_series_equal( - ops_mapping["round"](df1.a).compute(), pd.Series([1, 2, 3]), check_names=False, + ops_mapping["round"](df1.a).compute(), + pd.Series([1, 2, 3]), + check_names=False, ) assert_series_equal( ops_mapping["floor"](df1.a).compute(), diff --git a/tests/unit/test_context.py b/tests/unit/test_context.py index b84f9fa11..697c0aee5 100644 --- a/tests/unit/test_context.py +++ b/tests/unit/test_context.py @@ -88,7 +88,14 @@ def test_explain(gpu): @pytest.mark.parametrize( - "gpu", [False, pytest.param(True, marks=pytest.mark.gpu,),], + "gpu", + [ + False, + pytest.param( + True, + marks=pytest.mark.gpu, + ), + ], ) def test_sql(gpu): c = Context() @@ -112,7 +119,14 @@ def test_sql(gpu): @pytest.mark.parametrize( - "gpu", [False, pytest.param(True, marks=pytest.mark.gpu,),], + "gpu", + [ + False, + pytest.param( + True, + marks=pytest.mark.gpu, + ), + ], ) def test_input_types(temporary_data_file, gpu): c = Context() @@ -151,7 +165,11 @@ def assert_correct_output(gpu): @pytest.mark.parametrize( - "gpu", [False, pytest.param(True, marks=pytest.mark.gpu),], + "gpu", + [ + False, + pytest.param(True, marks=pytest.mark.gpu), + ], ) def test_tables_from_stack(gpu): c = Context() diff --git a/tests/unit/test_utils.py b/tests/unit/test_utils.py index 9016c0330..5b2df6563 100644 --- a/tests/unit/test_utils.py +++ b/tests/unit/test_utils.py @@ -70,7 +70,8 @@ def test_exception_parsing(): assert str(e) == expected e = ParsingException( - "SELECT * FROM df", """Lexical error at line 1, column 3. Message""", + "SELECT * FROM df", + """Lexical error at line 1, column 3. Message""", ) expected = """Can not parse the given SQL: Lexical error at line 1, column 3. Message @@ -97,7 +98,10 @@ def test_exception_parsing(): \tWHERE x = 3""" assert str(e) == expected - e = ParsingException("SELECT *", "Message",) + e = ParsingException( + "SELECT *", + "Message", + ) assert str(e) == "Message" From ab2aa5ab929c3ada7f0f1062a6c82c0d2ff0a3f1 Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Tue, 5 Apr 2022 12:36:07 -0400 Subject: [PATCH 05/44] Check for ucx-py nightlies when updating gpuCI (#441) * Simplify gpuCI updating workflow * Add check for cuML nightly version --- .github/workflows/update-gpuci.yml | 59 +++++++++++++++++++++++------- 1 file changed, 45 insertions(+), 14 deletions(-) diff --git a/.github/workflows/update-gpuci.yml b/.github/workflows/update-gpuci.yml index e3c9c8469..c98f46eba 100644 --- a/.github/workflows/update-gpuci.yml +++ b/.github/workflows/update-gpuci.yml @@ -13,39 +13,70 @@ jobs: steps: - uses: actions/checkout@v2 + - name: Parse current axis YAML + uses: the-coding-turtle/ga-yaml-parser@v0.1.1 + with: + file: continuous_integration/gpuci/axis.yaml + - name: Get latest cuDF nightly version - id: latest_version + id: cudf_latest uses: jacobtomlinson/gha-anaconda-package-version@0.1.3 with: org: "rapidsai-nightly" package: "cudf" version_system: "CalVer" - - name: Strip git tags from versions + - name: Get latest cuML nightly version + id: cuml_latest + uses: jacobtomlinson/gha-anaconda-package-version@0.1.3 + with: + org: "rapidsai-nightly" + package: "cuml" + version_system: "CalVer" + + - name: Get latest UCX-Py nightly version + id: ucx_py_latest + uses: jacobtomlinson/gha-anaconda-package-version@0.1.3 + with: + org: "rapidsai-nightly" + package: "ucx-py" + version_system: "CalVer" + + - name: Get old RAPIDS / UCX-Py versions env: - FULL_RAPIDS_VER: ${{ steps.latest_version.outputs.version }} - run: echo "RAPIDS_VER=${FULL_RAPIDS_VER::-10}" >> $GITHUB_ENV + FULL_CUDF_VER: ${{ steps.cudf_latest.outputs.version }} + FULL_CUML_VER: ${{ steps.cuml_latest.outputs.version }} + FULL_UCX_PY_VER: ${{ steps.ucx_py_latest.outputs.version }} + run: | + echo RAPIDS_VER=$RAPIDS_VER_0 >> $GITHUB_ENV + echo UCX_PY_VER=$(curl -sL https://version.gpuci.io/rapids/$RAPIDS_VER_0) >> $GITHUB_ENV + echo NEW_CUDF_VER=${FULL_CUDF_VER::-10} >> $GITHUB_ENV + echo NEW_CUML_VER=${FULL_CUML_VER::-10} >> $GITHUB_ENV + echo NEW_UCX_PY_VER=${FULL_UCX_PY_VER::-10} >> $GITHUB_ENV - - name: Find and Replace Release - uses: jacobtomlinson/gha-find-replace@0.1.4 + - name: Update RAPIDS version + uses: jacobtomlinson/gha-find-replace@2 with: include: 'continuous_integration\/gpuci\/axis\.yaml' - find: "RAPIDS_VER:\n- .*" - replace: |- - RAPIDS_VER: - - "${{ env.RAPIDS_VER }}" + find: "${{ env.RAPIDS_VER }}" + replace: "${{ env.NEW_CUDF_VER }}" + regex: false - name: Create Pull Request uses: peter-evans/create-pull-request@v3 + # make sure ucx-py nightlies are available and that cuDF/cuML nightly versions match up + if: | + env.UCX_PY_VER != env.NEW_UCX_PY_VER && + env.NEW_CUDF_VER == env.NEW_CUML_VER with: token: ${{ secrets.GITHUB_TOKEN }} draft: true - commit-message: "Update gpuCI `RAPIDS_VER` to `${{ env.RAPIDS_VER }}`" - title: "Update gpuCI `RAPIDS_VER` to `${{ env.RAPIDS_VER }}`" + commit-message: "Update gpuCI `RAPIDS_VER` to `${{ env.NEW_CUDF_VER }}`" + title: "Update gpuCI `RAPIDS_VER` to `${{ env.NEW_CUDF_VER }}`" team-reviewers: "dask/gpu" author: github-actions[bot] <41898282+github-actions[bot]@users.noreply.github.com> branch: "upgrade-gpuci-rapids" body: | - A new cuDF nightly version has been detected. + New cuDF and ucx-py nightly versions have been detected. - Updated `axis.yaml` to use `${{ env.RAPIDS_VER }}`. + Updated `axis.yaml` to use `${{ env.NEW_CUDF_VER }}`. From a28f757d865081ee44bc28d21a2a9ef7e8b3be70 Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Wed, 6 Apr 2022 17:06:31 -0400 Subject: [PATCH 06/44] Add handling for newer `prompt_toolkit` versions in cmd tests (#447) * Add handling for newer prompt-toolkit version * Place compatibility code in _compat --- dask_sql/_compat.py | 6 ++++++ tests/integration/test_cmd.py | 19 +++++++++++++------ 2 files changed, 19 insertions(+), 6 deletions(-) diff --git a/dask_sql/_compat.py b/dask_sql/_compat.py index 126416391..ee6d88c2f 100644 --- a/dask_sql/_compat.py +++ b/dask_sql/_compat.py @@ -1,6 +1,12 @@ import pandas as pd +import prompt_toolkit from packaging.version import parse as parseVersion _pandas_version = parseVersion(pd.__version__) +_prompt_toolkit_version = parseVersion(prompt_toolkit.__version__) + FLOAT_NAN_IMPLEMENTED = _pandas_version >= parseVersion("1.2.0") INT_NAN_IMPLEMENTED = _pandas_version >= parseVersion("1.0.0") + +# TODO: remove if prompt-toolkit min version gets bumped +PIPE_INPUT_CONTEXT_MANAGER = _prompt_toolkit_version >= parseVersion("3.0.29") diff --git a/tests/integration/test_cmd.py b/tests/integration/test_cmd.py index 145b4962f..fdef31bd3 100644 --- a/tests/integration/test_cmd.py +++ b/tests/integration/test_cmd.py @@ -6,17 +6,24 @@ from prompt_toolkit.output import DummyOutput from prompt_toolkit.shortcuts import PromptSession +from dask_sql._compat import PIPE_INPUT_CONTEXT_MANAGER from dask_sql.cmd import _meta_commands @pytest.fixture(autouse=True, scope="function") def mock_prompt_input(): - pipe_input = create_pipe_input() - try: - with create_app_session(input=pipe_input, output=DummyOutput()): - yield pipe_input - finally: - pipe_input.close() + # TODO: remove if prompt-toolkit min version gets bumped + if PIPE_INPUT_CONTEXT_MANAGER: + with create_pipe_input() as pipe_input: + with create_app_session(input=pipe_input, output=DummyOutput()): + yield pipe_input + else: + pipe_input = create_pipe_input() + try: + with create_app_session(input=pipe_input, output=DummyOutput()): + yield pipe_input + finally: + pipe_input.close() def _feed_cli_with_input( From 486fc669fbb93b12ab83e9b0a6846d63f6d63b80 Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Wed, 6 Apr 2022 17:06:49 -0400 Subject: [PATCH 07/44] Fix version for gha-find-replace (#446) --- .github/workflows/update-gpuci.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/update-gpuci.yml b/.github/workflows/update-gpuci.yml index c98f46eba..1cf91c069 100644 --- a/.github/workflows/update-gpuci.yml +++ b/.github/workflows/update-gpuci.yml @@ -55,7 +55,7 @@ jobs: echo NEW_UCX_PY_VER=${FULL_UCX_PY_VER::-10} >> $GITHUB_ENV - name: Update RAPIDS version - uses: jacobtomlinson/gha-find-replace@2 + uses: jacobtomlinson/gha-find-replace@v2 with: include: 'continuous_integration\/gpuci\/axis\.yaml' find: "${{ env.RAPIDS_VER }}" From ce176e0f85b40293b9b236bd143f7ffe6dc0b6c8 Mon Sep 17 00:00:00 2001 From: Ayush Dattagupta Date: Thu, 7 Apr 2022 09:24:30 -0700 Subject: [PATCH 08/44] Update versions of Java dependencies (#445) * Update versions for java dependencies with cves * Rerun tests --- planner/pom.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/planner/pom.xml b/planner/pom.xml index 817827cf0..ce1f1e3e5 100755 --- a/planner/pom.xml +++ b/planner/pom.xml @@ -95,7 +95,7 @@ com.fasterxml.jackson.dataformat jackson-dataformat-yaml - 2.10.0 + 2.13.2 runtime @@ -112,7 +112,7 @@ com.jayway.jsonpath json-path - 2.4.0 + 2.7.0 com.yahoo.datasketches @@ -142,7 +142,7 @@ commons-io commons-io - 2.4 + 2.11.0 runtime From 50d95d25bcef6f9d2237a92b6d8bf78832bf1c81 Mon Sep 17 00:00:00 2001 From: Ayush Dattagupta Date: Thu, 7 Apr 2022 10:33:29 -0700 Subject: [PATCH 09/44] Update jackson databind version (#449) * Update versions for java dependencies with cves * Rerun tests * update jackson-databind dependency --- planner/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/planner/pom.xml b/planner/pom.xml index ce1f1e3e5..2cdb09240 100755 --- a/planner/pom.xml +++ b/planner/pom.xml @@ -90,7 +90,7 @@ com.fasterxml.jackson.core jackson-databind - 2.10.0 + 2.13.2.2 com.fasterxml.jackson.dataformat From 37a3a61fb13b0c56fcc10bf8ef01f4885a58dae8 Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Thu, 7 Apr 2022 14:05:47 -0400 Subject: [PATCH 10/44] Disable SQL server functionality (#448) * Disable SQL server functionality * Update docs/source/server.rst Co-authored-by: Ayush Dattagupta * Disable server at lowest possible level * Skip all server tests * Add tests to ensure server is disabled * Fix CVE fix test Co-authored-by: Ayush Dattagupta --- continuous_integration/recipe/meta.yaml | 6 ++- dask_sql/__init__.py | 7 ++- dask_sql/context.py | 2 +- dask_sql/server/app.py | 3 ++ docs/source/server.rst | 4 ++ setup.py | 3 +- tests/integration/test_cve_fix.py | 16 +++++++ tests/integration/test_jdbc.py | 5 ++ tests/integration/test_server.py | 63 +++++++++++++------------ 9 files changed, 74 insertions(+), 35 deletions(-) create mode 100644 tests/integration/test_cve_fix.py diff --git a/continuous_integration/recipe/meta.yaml b/continuous_integration/recipe/meta.yaml index a56bd15c2..15d8e37ee 100644 --- a/continuous_integration/recipe/meta.yaml +++ b/continuous_integration/recipe/meta.yaml @@ -15,7 +15,8 @@ build: number: {{ GIT_DESCRIBE_NUMBER }} noarch: python entry_points: - - dask-sql-server = dask_sql.server.app:main + # TODO: re-enable server once CVEs are resolved + # - dask-sql-server = dask_sql.server.app:main - dask-sql = dask_sql.cmd:main string: py_{{ GIT_DESCRIBE_HASH }}_{{ GIT_DESCRIBE_NUMBER }} script: {{ PYTHON }} -m pip install . --no-deps -vv @@ -45,7 +46,8 @@ test: - dask_sql commands: - pip check - - dask-sql-server --help + # TODO: re-enable server once CVEs are resolved + # - dask-sql-server --help - dask-sql --help requires: - pip diff --git a/dask_sql/__init__.py b/dask_sql/__init__.py index d343a4c5c..96a70d873 100644 --- a/dask_sql/__init__.py +++ b/dask_sql/__init__.py @@ -3,9 +3,12 @@ from .cmd import cmd_loop from .context import Context from .datacontainer import Statistics -from .server.app import run_server + +# from .server.app import run_server __version__ = get_versions()["version"] del get_versions -__all__ = [__version__, cmd_loop, Context, run_server, Statistics] +# TODO: re-enable server once CVEs are resolved +# __all__ = [__version__, cmd_loop, Context, run_server, Statistics] +__all__ = [__version__, cmd_loop, Context, Statistics] diff --git a/dask_sql/context.py b/dask_sql/context.py index 98cc46e21..0449f4835 100644 --- a/dask_sql/context.py +++ b/dask_sql/context.py @@ -659,7 +659,7 @@ def run_server( from dask_sql.server.app import run_server self.stop_server() - self.server = run_server( + self.sql_server = run_server( context=self, client=client, host=host, diff --git a/dask_sql/server/app.py b/dask_sql/server/app.py index 634de3856..261f75831 100644 --- a/dask_sql/server/app.py +++ b/dask_sql/server/app.py @@ -276,6 +276,9 @@ def _init_app( context: Context = None, client: dask.distributed.Client = None, ): + # TODO: re-enable server once CVEs are resolved + raise NotImplementedError + app.c = context or Context() app.future_list = {} diff --git a/docs/source/server.rst b/docs/source/server.rst index 70ad902e9..8993d55fb 100644 --- a/docs/source/server.rst +++ b/docs/source/server.rst @@ -3,6 +3,10 @@ SQL Server ========== +.. warning:: + + ``dask-sql``'s SQL server functionality is currently exploitable and has been disabled until the exposed vulnerabilities can be resolved. + ``dask-sql`` comes with a small test implementation for a SQL server. Instead of rebuilding a full ODBC driver, we re-use the `presto wire protocol `_. diff --git a/setup.py b/setup.py index e63b9a3d5..98781fc8c 100755 --- a/setup.py +++ b/setup.py @@ -116,7 +116,8 @@ def build(self): }, entry_points={ "console_scripts": [ - "dask-sql-server = dask_sql.server.app:main", + # TODO: re-enable server once CVEs are resolved + # "dask-sql-server = dask_sql.server.app:main", "dask-sql = dask_sql.cmd:main", ] }, diff --git a/tests/integration/test_cve_fix.py b/tests/integration/test_cve_fix.py new file mode 100644 index 000000000..87ec03bf1 --- /dev/null +++ b/tests/integration/test_cve_fix.py @@ -0,0 +1,16 @@ +import pytest + +from dask_sql import Context +from dask_sql.server.app import _init_app, app + + +def test_run_server_disabled(c): + with pytest.raises(NotImplementedError): + c.run_server() + + +def test_init_app_disabled(): + c = Context() + c.sql("SELECT 1 + 1").compute() + with pytest.raises(NotImplementedError): + _init_app(app, c) diff --git a/tests/integration/test_jdbc.py b/tests/integration/test_jdbc.py index f8426ae46..2f6eb464b 100644 --- a/tests/integration/test_jdbc.py +++ b/tests/integration/test_jdbc.py @@ -7,6 +7,11 @@ from dask_sql.server.app import _init_app, app from dask_sql.server.presto_jdbc import create_meta_data +# TODO: re-enable server once CVEs are resolved +pytest.skip( + "SQL server is disabled until related CVEs are resolved", allow_module_level=True +) + # needed for the testclient pytest.importorskip("requests") diff --git a/tests/integration/test_server.py b/tests/integration/test_server.py index 88d08a4f7..c5c460903 100644 --- a/tests/integration/test_server.py +++ b/tests/integration/test_server.py @@ -5,6 +5,11 @@ from dask_sql import Context from dask_sql.server.app import _init_app, app +# TODO: re-enable server once CVEs are resolved +pytest.skip( + "SQL server is disabled until related CVEs are resolved", allow_module_level=True +) + # needed for the testclient pytest.importorskip("requests") @@ -23,6 +28,35 @@ def app_client(): app.client.close() +def get_result_or_error(app_client, response): + result = response.json() + + assert "nextUri" in result + assert "error" not in result + + status_url = result["nextUri"] + next_url = status_url + + counter = 0 + while True: + response = app_client.get(next_url) + assert response.status_code == 200 + + result = response.json() + + if "nextUri" not in result: + break + + next_url = result["nextUri"] + + counter += 1 + assert counter <= 100 + + sleep(0.1) + + return result + + def test_routes(app_client): assert app_client.post("/v1/statement", data="SELECT 1 + 1").status_code == 200 assert app_client.get("/v1/statement", data="SELECT 1 + 1").status_code == 405 @@ -174,32 +208,3 @@ def test_inf_table(app_client, user_table_inf): assert len(result["data"]) == 3 assert result["data"][1] == ["+Infinity"] assert "error" not in result - - -def get_result_or_error(app_client, response): - result = response.json() - - assert "nextUri" in result - assert "error" not in result - - status_url = result["nextUri"] - next_url = status_url - - counter = 0 - while True: - response = app_client.get(next_url) - assert response.status_code == 200 - - result = response.json() - - if "nextUri" not in result: - break - - next_url = result["nextUri"] - - counter += 1 - assert counter <= 100 - - sleep(0.1) - - return result From ffdc42ff31f98b6596c3163113a50ad820adc21d Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Thu, 7 Apr 2022 14:40:29 -0400 Subject: [PATCH 11/44] Update dask pinnings for release (#450) --- continuous_integration/environment-3.10-jdk11-dev.yaml | 2 +- continuous_integration/environment-3.10-jdk8-dev.yaml | 2 +- continuous_integration/environment-3.8-jdk11-dev.yaml | 2 +- continuous_integration/environment-3.8-jdk8-dev.yaml | 2 +- continuous_integration/environment-3.9-jdk11-dev.yaml | 2 +- continuous_integration/environment-3.9-jdk8-dev.yaml | 2 +- continuous_integration/recipe/meta.yaml | 2 +- docker/conda.txt | 2 +- docs/environment.yml | 2 +- docs/requirements-docs.txt | 2 +- setup.py | 2 +- 11 files changed, 11 insertions(+), 11 deletions(-) diff --git a/continuous_integration/environment-3.10-jdk11-dev.yaml b/continuous_integration/environment-3.10-jdk11-dev.yaml index eae7fc83d..f82f8f31c 100644 --- a/continuous_integration/environment-3.10-jdk11-dev.yaml +++ b/continuous_integration/environment-3.10-jdk11-dev.yaml @@ -8,7 +8,7 @@ dependencies: - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 -- dask>=2021.11.1 +- dask>=2022.3.0,<2022.4.1 - fastapi>=0.61.1 - fs>=2.4.11 - intake>=0.6.0 diff --git a/continuous_integration/environment-3.10-jdk8-dev.yaml b/continuous_integration/environment-3.10-jdk8-dev.yaml index 59183a513..4a836dd7f 100644 --- a/continuous_integration/environment-3.10-jdk8-dev.yaml +++ b/continuous_integration/environment-3.10-jdk8-dev.yaml @@ -8,7 +8,7 @@ dependencies: - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 -- dask>=2021.11.1 +- dask>=2022.3.0,<2022.4.1 - fastapi>=0.61.1 - fs>=2.4.11 - intake>=0.6.0 diff --git a/continuous_integration/environment-3.8-jdk11-dev.yaml b/continuous_integration/environment-3.8-jdk11-dev.yaml index 1a00258f7..cfca7d119 100644 --- a/continuous_integration/environment-3.8-jdk11-dev.yaml +++ b/continuous_integration/environment-3.8-jdk11-dev.yaml @@ -8,7 +8,7 @@ dependencies: - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 -- dask>=2021.11.1 +- dask>=2022.3.0,<2022.4.1 - fastapi>=0.61.1 - fs>=2.4.11 - intake>=0.6.0 diff --git a/continuous_integration/environment-3.8-jdk8-dev.yaml b/continuous_integration/environment-3.8-jdk8-dev.yaml index 9aaf4cd95..b98213e0c 100644 --- a/continuous_integration/environment-3.8-jdk8-dev.yaml +++ b/continuous_integration/environment-3.8-jdk8-dev.yaml @@ -8,7 +8,7 @@ dependencies: - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 -- dask>=2021.11.1 +- dask>=2022.3.0,<2022.4.1 - fastapi>=0.61.1 - fs>=2.4.11 - intake>=0.6.0 diff --git a/continuous_integration/environment-3.9-jdk11-dev.yaml b/continuous_integration/environment-3.9-jdk11-dev.yaml index df68fedd4..1c476937b 100644 --- a/continuous_integration/environment-3.9-jdk11-dev.yaml +++ b/continuous_integration/environment-3.9-jdk11-dev.yaml @@ -8,7 +8,7 @@ dependencies: - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 -- dask>=2021.11.1 +- dask>=2022.3.0,<2022.4.1 - fastapi>=0.61.1 - fs>=2.4.11 - intake>=0.6.0 diff --git a/continuous_integration/environment-3.9-jdk8-dev.yaml b/continuous_integration/environment-3.9-jdk8-dev.yaml index f27acff9c..7dd3b0afa 100644 --- a/continuous_integration/environment-3.9-jdk8-dev.yaml +++ b/continuous_integration/environment-3.9-jdk8-dev.yaml @@ -8,7 +8,7 @@ dependencies: - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 -- dask>=2021.11.1 +- dask>=2022.3.0,<2022.4.1 - fastapi>=0.61.1 - fs>=2.4.11 - intake>=0.6.0 diff --git a/continuous_integration/recipe/meta.yaml b/continuous_integration/recipe/meta.yaml index 15d8e37ee..518c167bb 100644 --- a/continuous_integration/recipe/meta.yaml +++ b/continuous_integration/recipe/meta.yaml @@ -29,7 +29,7 @@ requirements: - python >=3.8 run: - python - - dask >=2021.11.1 + - dask >=2022.3.0,<2022.4.1 - pandas >=1.0.0 - jpype1 >=1.0.2 - openjdk >=8 diff --git a/docker/conda.txt b/docker/conda.txt index d833ac877..0eaae245b 100644 --- a/docker/conda.txt +++ b/docker/conda.txt @@ -1,5 +1,5 @@ python>=3.8 -dask>=2021.11.1 +dask>=2022.3.0,<2022.4.1 pandas>=1.0.0 # below 1.0, there were no nullable ext. types jpype1>=1.0.2 openjdk>=8 diff --git a/docs/environment.yml b/docs/environment.yml index e67ec890f..1371863a7 100644 --- a/docs/environment.yml +++ b/docs/environment.yml @@ -8,7 +8,7 @@ dependencies: - sphinx-tabs - dask-sphinx-theme>=2.0.3 - maven>=3.6.0 - - dask>=2021.11.1 + - dask>=2022.3.0,<2022.4.1 - pandas>=1.0.0 - fugue>=0.5.3 - jpype1>=1.0.2 diff --git a/docs/requirements-docs.txt b/docs/requirements-docs.txt index 7d9e2e6cc..dbfc08369 100644 --- a/docs/requirements-docs.txt +++ b/docs/requirements-docs.txt @@ -1,7 +1,7 @@ sphinx>=4.0.0 sphinx-tabs dask-sphinx-theme>=2.0.3 -dask>=2021.11.1 +dask>=2022.3.0,<2022.4.1 pandas>=1.0.0 fugue>=0.5.3 jpype1>=1.0.2 diff --git a/setup.py b/setup.py index 98781fc8c..d8bbabfb4 100755 --- a/setup.py +++ b/setup.py @@ -87,7 +87,7 @@ def build(self): python_requires=">=3.8", setup_requires=sphinx_requirements, install_requires=[ - "dask[dataframe,distributed]>=2021.11.1", + "dask[dataframe,distributed]>=2022.3.0,<2022.4.1", "pandas>=1.0.0", # below 1.0, there were no nullable ext. types "jpype1>=1.0.2", "fastapi>=0.61.1", From fa74aef18b8fa3cef1f1f1cece28f34b6bcb2856 Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Thu, 7 Apr 2022 16:49:08 -0400 Subject: [PATCH 12/44] Add Java source code to source distribution (#451) --- MANIFEST.in | 1 + 1 file changed, 1 insertion(+) diff --git a/MANIFEST.in b/MANIFEST.in index 2b6351550..4b4310eee 100644 --- a/MANIFEST.in +++ b/MANIFEST.in @@ -1,4 +1,5 @@ recursive-include dask_sql *.yaml +recursive-include planner * include versioneer.py include dask_sql/_version.py From 37ea6b6c0d87ff9092fd0fd04610f5ea8683af74 Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Fri, 8 Apr 2022 13:33:10 -0400 Subject: [PATCH 13/44] Bump `httpclient` dependency (#453) * Revert "Disable SQL server functionality (#448)" This reverts commit 37a3a61fb13b0c56fcc10bf8ef01f4885a58dae8. * Bump httpclient version --- continuous_integration/recipe/meta.yaml | 6 +-- dask_sql/__init__.py | 7 +-- dask_sql/context.py | 2 +- dask_sql/server/app.py | 3 -- docs/source/server.rst | 4 -- planner/pom.xml | 5 ++ setup.py | 3 +- tests/integration/test_cve_fix.py | 16 ------- tests/integration/test_jdbc.py | 5 -- tests/integration/test_server.py | 63 ++++++++++++------------- 10 files changed, 40 insertions(+), 74 deletions(-) delete mode 100644 tests/integration/test_cve_fix.py diff --git a/continuous_integration/recipe/meta.yaml b/continuous_integration/recipe/meta.yaml index 518c167bb..8ff12cdf4 100644 --- a/continuous_integration/recipe/meta.yaml +++ b/continuous_integration/recipe/meta.yaml @@ -15,8 +15,7 @@ build: number: {{ GIT_DESCRIBE_NUMBER }} noarch: python entry_points: - # TODO: re-enable server once CVEs are resolved - # - dask-sql-server = dask_sql.server.app:main + - dask-sql-server = dask_sql.server.app:main - dask-sql = dask_sql.cmd:main string: py_{{ GIT_DESCRIBE_HASH }}_{{ GIT_DESCRIBE_NUMBER }} script: {{ PYTHON }} -m pip install . --no-deps -vv @@ -46,8 +45,7 @@ test: - dask_sql commands: - pip check - # TODO: re-enable server once CVEs are resolved - # - dask-sql-server --help + - dask-sql-server --help - dask-sql --help requires: - pip diff --git a/dask_sql/__init__.py b/dask_sql/__init__.py index 96a70d873..d343a4c5c 100644 --- a/dask_sql/__init__.py +++ b/dask_sql/__init__.py @@ -3,12 +3,9 @@ from .cmd import cmd_loop from .context import Context from .datacontainer import Statistics - -# from .server.app import run_server +from .server.app import run_server __version__ = get_versions()["version"] del get_versions -# TODO: re-enable server once CVEs are resolved -# __all__ = [__version__, cmd_loop, Context, run_server, Statistics] -__all__ = [__version__, cmd_loop, Context, Statistics] +__all__ = [__version__, cmd_loop, Context, run_server, Statistics] diff --git a/dask_sql/context.py b/dask_sql/context.py index 0449f4835..98cc46e21 100644 --- a/dask_sql/context.py +++ b/dask_sql/context.py @@ -659,7 +659,7 @@ def run_server( from dask_sql.server.app import run_server self.stop_server() - self.sql_server = run_server( + self.server = run_server( context=self, client=client, host=host, diff --git a/dask_sql/server/app.py b/dask_sql/server/app.py index 261f75831..634de3856 100644 --- a/dask_sql/server/app.py +++ b/dask_sql/server/app.py @@ -276,9 +276,6 @@ def _init_app( context: Context = None, client: dask.distributed.Client = None, ): - # TODO: re-enable server once CVEs are resolved - raise NotImplementedError - app.c = context or Context() app.future_list = {} diff --git a/docs/source/server.rst b/docs/source/server.rst index 8993d55fb..70ad902e9 100644 --- a/docs/source/server.rst +++ b/docs/source/server.rst @@ -3,10 +3,6 @@ SQL Server ========== -.. warning:: - - ``dask-sql``'s SQL server functionality is currently exploitable and has been disabled until the exposed vulnerabilities can be resolved. - ``dask-sql`` comes with a small test implementation for a SQL server. Instead of rebuilding a full ODBC driver, we re-use the `presto wire protocol `_. diff --git a/planner/pom.xml b/planner/pom.xml index 2cdb09240..2d6825c3a 100755 --- a/planner/pom.xml +++ b/planner/pom.xml @@ -61,6 +61,11 @@ avatica-core 1.20.0 + + org.apache.httpcomponents + httpclient + 4.5.13 + org.apiguardian apiguardian-api diff --git a/setup.py b/setup.py index d8bbabfb4..844f52880 100755 --- a/setup.py +++ b/setup.py @@ -116,8 +116,7 @@ def build(self): }, entry_points={ "console_scripts": [ - # TODO: re-enable server once CVEs are resolved - # "dask-sql-server = dask_sql.server.app:main", + "dask-sql-server = dask_sql.server.app:main", "dask-sql = dask_sql.cmd:main", ] }, diff --git a/tests/integration/test_cve_fix.py b/tests/integration/test_cve_fix.py deleted file mode 100644 index 87ec03bf1..000000000 --- a/tests/integration/test_cve_fix.py +++ /dev/null @@ -1,16 +0,0 @@ -import pytest - -from dask_sql import Context -from dask_sql.server.app import _init_app, app - - -def test_run_server_disabled(c): - with pytest.raises(NotImplementedError): - c.run_server() - - -def test_init_app_disabled(): - c = Context() - c.sql("SELECT 1 + 1").compute() - with pytest.raises(NotImplementedError): - _init_app(app, c) diff --git a/tests/integration/test_jdbc.py b/tests/integration/test_jdbc.py index 2f6eb464b..f8426ae46 100644 --- a/tests/integration/test_jdbc.py +++ b/tests/integration/test_jdbc.py @@ -7,11 +7,6 @@ from dask_sql.server.app import _init_app, app from dask_sql.server.presto_jdbc import create_meta_data -# TODO: re-enable server once CVEs are resolved -pytest.skip( - "SQL server is disabled until related CVEs are resolved", allow_module_level=True -) - # needed for the testclient pytest.importorskip("requests") diff --git a/tests/integration/test_server.py b/tests/integration/test_server.py index c5c460903..88d08a4f7 100644 --- a/tests/integration/test_server.py +++ b/tests/integration/test_server.py @@ -5,11 +5,6 @@ from dask_sql import Context from dask_sql.server.app import _init_app, app -# TODO: re-enable server once CVEs are resolved -pytest.skip( - "SQL server is disabled until related CVEs are resolved", allow_module_level=True -) - # needed for the testclient pytest.importorskip("requests") @@ -28,35 +23,6 @@ def app_client(): app.client.close() -def get_result_or_error(app_client, response): - result = response.json() - - assert "nextUri" in result - assert "error" not in result - - status_url = result["nextUri"] - next_url = status_url - - counter = 0 - while True: - response = app_client.get(next_url) - assert response.status_code == 200 - - result = response.json() - - if "nextUri" not in result: - break - - next_url = result["nextUri"] - - counter += 1 - assert counter <= 100 - - sleep(0.1) - - return result - - def test_routes(app_client): assert app_client.post("/v1/statement", data="SELECT 1 + 1").status_code == 200 assert app_client.get("/v1/statement", data="SELECT 1 + 1").status_code == 405 @@ -208,3 +174,32 @@ def test_inf_table(app_client, user_table_inf): assert len(result["data"]) == 3 assert result["data"][1] == ["+Infinity"] assert "error" not in result + + +def get_result_or_error(app_client, response): + result = response.json() + + assert "nextUri" in result + assert "error" not in result + + status_url = result["nextUri"] + next_url = status_url + + counter = 0 + while True: + response = app_client.get(next_url) + assert response.status_code == 200 + + result = response.json() + + if "nextUri" not in result: + break + + next_url = result["nextUri"] + + counter += 1 + assert counter <= 100 + + sleep(0.1) + + return result From f19ee4d5fdfc03a1b50d2c4553eb79a5ba617284 Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Mon, 11 Apr 2022 11:34:18 -0400 Subject: [PATCH 14/44] Unpin Dask/distributed versions (#452) * Unpin dask/distributed post release * Remove dask/distributed version ceiling --- continuous_integration/environment-3.10-jdk11-dev.yaml | 2 +- continuous_integration/environment-3.10-jdk8-dev.yaml | 2 +- continuous_integration/environment-3.8-jdk11-dev.yaml | 2 +- continuous_integration/environment-3.8-jdk8-dev.yaml | 2 +- continuous_integration/environment-3.9-jdk11-dev.yaml | 2 +- continuous_integration/environment-3.9-jdk8-dev.yaml | 2 +- continuous_integration/recipe/meta.yaml | 2 +- docker/conda.txt | 2 +- docs/environment.yml | 2 +- docs/requirements-docs.txt | 2 +- setup.py | 2 +- 11 files changed, 11 insertions(+), 11 deletions(-) diff --git a/continuous_integration/environment-3.10-jdk11-dev.yaml b/continuous_integration/environment-3.10-jdk11-dev.yaml index f82f8f31c..85a25a2f7 100644 --- a/continuous_integration/environment-3.10-jdk11-dev.yaml +++ b/continuous_integration/environment-3.10-jdk11-dev.yaml @@ -8,7 +8,7 @@ dependencies: - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 -- dask>=2022.3.0,<2022.4.1 +- dask>=2022.3.0 - fastapi>=0.61.1 - fs>=2.4.11 - intake>=0.6.0 diff --git a/continuous_integration/environment-3.10-jdk8-dev.yaml b/continuous_integration/environment-3.10-jdk8-dev.yaml index 4a836dd7f..c543f6b14 100644 --- a/continuous_integration/environment-3.10-jdk8-dev.yaml +++ b/continuous_integration/environment-3.10-jdk8-dev.yaml @@ -8,7 +8,7 @@ dependencies: - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 -- dask>=2022.3.0,<2022.4.1 +- dask>=2022.3.0 - fastapi>=0.61.1 - fs>=2.4.11 - intake>=0.6.0 diff --git a/continuous_integration/environment-3.8-jdk11-dev.yaml b/continuous_integration/environment-3.8-jdk11-dev.yaml index cfca7d119..d77245681 100644 --- a/continuous_integration/environment-3.8-jdk11-dev.yaml +++ b/continuous_integration/environment-3.8-jdk11-dev.yaml @@ -8,7 +8,7 @@ dependencies: - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 -- dask>=2022.3.0,<2022.4.1 +- dask>=2022.3.0 - fastapi>=0.61.1 - fs>=2.4.11 - intake>=0.6.0 diff --git a/continuous_integration/environment-3.8-jdk8-dev.yaml b/continuous_integration/environment-3.8-jdk8-dev.yaml index b98213e0c..63c9528d6 100644 --- a/continuous_integration/environment-3.8-jdk8-dev.yaml +++ b/continuous_integration/environment-3.8-jdk8-dev.yaml @@ -8,7 +8,7 @@ dependencies: - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 -- dask>=2022.3.0,<2022.4.1 +- dask>=2022.3.0 - fastapi>=0.61.1 - fs>=2.4.11 - intake>=0.6.0 diff --git a/continuous_integration/environment-3.9-jdk11-dev.yaml b/continuous_integration/environment-3.9-jdk11-dev.yaml index 1c476937b..6923ebab8 100644 --- a/continuous_integration/environment-3.9-jdk11-dev.yaml +++ b/continuous_integration/environment-3.9-jdk11-dev.yaml @@ -8,7 +8,7 @@ dependencies: - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 -- dask>=2022.3.0,<2022.4.1 +- dask>=2022.3.0 - fastapi>=0.61.1 - fs>=2.4.11 - intake>=0.6.0 diff --git a/continuous_integration/environment-3.9-jdk8-dev.yaml b/continuous_integration/environment-3.9-jdk8-dev.yaml index 7dd3b0afa..4404b5f8a 100644 --- a/continuous_integration/environment-3.9-jdk8-dev.yaml +++ b/continuous_integration/environment-3.9-jdk8-dev.yaml @@ -8,7 +8,7 @@ dependencies: - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 -- dask>=2022.3.0,<2022.4.1 +- dask>=2022.3.0 - fastapi>=0.61.1 - fs>=2.4.11 - intake>=0.6.0 diff --git a/continuous_integration/recipe/meta.yaml b/continuous_integration/recipe/meta.yaml index 8ff12cdf4..b2a40190a 100644 --- a/continuous_integration/recipe/meta.yaml +++ b/continuous_integration/recipe/meta.yaml @@ -28,7 +28,7 @@ requirements: - python >=3.8 run: - python - - dask >=2022.3.0,<2022.4.1 + - dask >=2022.3.0 - pandas >=1.0.0 - jpype1 >=1.0.2 - openjdk >=8 diff --git a/docker/conda.txt b/docker/conda.txt index 0eaae245b..1796da131 100644 --- a/docker/conda.txt +++ b/docker/conda.txt @@ -1,5 +1,5 @@ python>=3.8 -dask>=2022.3.0,<2022.4.1 +dask>=2022.3.0 pandas>=1.0.0 # below 1.0, there were no nullable ext. types jpype1>=1.0.2 openjdk>=8 diff --git a/docs/environment.yml b/docs/environment.yml index 1371863a7..ffd29270b 100644 --- a/docs/environment.yml +++ b/docs/environment.yml @@ -8,7 +8,7 @@ dependencies: - sphinx-tabs - dask-sphinx-theme>=2.0.3 - maven>=3.6.0 - - dask>=2022.3.0,<2022.4.1 + - dask>=2022.3.0 - pandas>=1.0.0 - fugue>=0.5.3 - jpype1>=1.0.2 diff --git a/docs/requirements-docs.txt b/docs/requirements-docs.txt index dbfc08369..0cc941db8 100644 --- a/docs/requirements-docs.txt +++ b/docs/requirements-docs.txt @@ -1,7 +1,7 @@ sphinx>=4.0.0 sphinx-tabs dask-sphinx-theme>=2.0.3 -dask>=2022.3.0,<2022.4.1 +dask>=2022.3.0 pandas>=1.0.0 fugue>=0.5.3 jpype1>=1.0.2 diff --git a/setup.py b/setup.py index 844f52880..f3b59bba8 100755 --- a/setup.py +++ b/setup.py @@ -87,7 +87,7 @@ def build(self): python_requires=">=3.8", setup_requires=sphinx_requirements, install_requires=[ - "dask[dataframe,distributed]>=2022.3.0,<2022.4.1", + "dask[dataframe,distributed]>=2022.3.0", "pandas>=1.0.0", # below 1.0, there were no nullable ext. types "jpype1>=1.0.2", "fastapi>=0.61.1", From 1eb30c1cbce83c918723ec6a33fd867148183c46 Mon Sep 17 00:00:00 2001 From: Ayush Dattagupta Date: Mon, 11 Apr 2022 13:35:15 -0700 Subject: [PATCH 15/44] Add jsonschema to ci testing (#454) * Add jsonschema to ci env * Fix typo in config schema --- continuous_integration/environment-3.10-jdk11-dev.yaml | 1 + continuous_integration/environment-3.10-jdk8-dev.yaml | 1 + continuous_integration/environment-3.8-jdk11-dev.yaml | 1 + continuous_integration/environment-3.8-jdk8-dev.yaml | 1 + continuous_integration/environment-3.9-jdk11-dev.yaml | 1 + continuous_integration/environment-3.9-jdk8-dev.yaml | 1 + dask_sql/sql-schema.yaml | 2 +- 7 files changed, 7 insertions(+), 1 deletion(-) diff --git a/continuous_integration/environment-3.10-jdk11-dev.yaml b/continuous_integration/environment-3.10-jdk11-dev.yaml index 85a25a2f7..257b33703 100644 --- a/continuous_integration/environment-3.10-jdk11-dev.yaml +++ b/continuous_integration/environment-3.10-jdk11-dev.yaml @@ -14,6 +14,7 @@ dependencies: - intake>=0.6.0 - isort=5.7.0 - jpype1>=1.0.2 +- jsonschema>=4.4.0 - lightgbm>=3.2.1 - maven>=3.6.0 - mlflow>=1.19.0 diff --git a/continuous_integration/environment-3.10-jdk8-dev.yaml b/continuous_integration/environment-3.10-jdk8-dev.yaml index c543f6b14..399aeb53d 100644 --- a/continuous_integration/environment-3.10-jdk8-dev.yaml +++ b/continuous_integration/environment-3.10-jdk8-dev.yaml @@ -14,6 +14,7 @@ dependencies: - intake>=0.6.0 - isort=5.7.0 - jpype1>=1.0.2 +- jsonschema>=4.4.0 - lightgbm>=3.2.1 - maven>=3.6.0 - mlflow>=1.19.0 diff --git a/continuous_integration/environment-3.8-jdk11-dev.yaml b/continuous_integration/environment-3.8-jdk11-dev.yaml index d77245681..03968298a 100644 --- a/continuous_integration/environment-3.8-jdk11-dev.yaml +++ b/continuous_integration/environment-3.8-jdk11-dev.yaml @@ -14,6 +14,7 @@ dependencies: - intake>=0.6.0 - isort=5.7.0 - jpype1>=1.0.2 +- jsonschema>=4.4.0 - lightgbm>=3.2.1 - maven>=3.6.0 - mlflow>=1.19.0 diff --git a/continuous_integration/environment-3.8-jdk8-dev.yaml b/continuous_integration/environment-3.8-jdk8-dev.yaml index 63c9528d6..46c77c06a 100644 --- a/continuous_integration/environment-3.8-jdk8-dev.yaml +++ b/continuous_integration/environment-3.8-jdk8-dev.yaml @@ -14,6 +14,7 @@ dependencies: - intake>=0.6.0 - isort=5.7.0 - jpype1>=1.0.2 +- jsonschema>=4.4.0 - lightgbm>=3.2.1 - maven>=3.6.0 - mlflow>=1.19.0 diff --git a/continuous_integration/environment-3.9-jdk11-dev.yaml b/continuous_integration/environment-3.9-jdk11-dev.yaml index 6923ebab8..b3b2d0fa4 100644 --- a/continuous_integration/environment-3.9-jdk11-dev.yaml +++ b/continuous_integration/environment-3.9-jdk11-dev.yaml @@ -14,6 +14,7 @@ dependencies: - intake>=0.6.0 - isort=5.7.0 - jpype1>=1.0.2 +- jsonschema>=4.4.0 - lightgbm>=3.2.1 - maven>=3.6.0 - mlflow>=1.19.0 diff --git a/continuous_integration/environment-3.9-jdk8-dev.yaml b/continuous_integration/environment-3.9-jdk8-dev.yaml index 4404b5f8a..7afce7d05 100644 --- a/continuous_integration/environment-3.9-jdk8-dev.yaml +++ b/continuous_integration/environment-3.9-jdk8-dev.yaml @@ -14,6 +14,7 @@ dependencies: - intake>=0.6.0 - isort=5.7.0 - jpype1>=1.0.2 +- jsonschema>=4.4.0 - lightgbm>=3.2.1 - maven>=3.6.0 - mlflow>=1.19.0 diff --git a/dask_sql/sql-schema.yaml b/dask_sql/sql-schema.yaml index f65e4d344..929ab1e0b 100644 --- a/dask_sql/sql-schema.yaml +++ b/dask_sql/sql-schema.yaml @@ -28,6 +28,6 @@ properties: Whether sql identifiers are considered case sensitive while parsing. predicate_pushdown: - type: bool + type: boolean description: | Whether to try pushing down filter predicates into IO (when possible). From 2bd1d186004ff6516c84534899051e1bda952add Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Mon, 11 Apr 2022 17:00:35 -0400 Subject: [PATCH 16/44] Switch tests from `pd.testing.assert_frame_equal` to `dd.assert_eq` (#365) * Start moving tests to dd.assert_eq * Use assert_eq in datetime filter test * Resolve most resulting test failures * Resolve remaining test failures * Convert over tests * Convert more tests * Consolidate select limit cpu/gpu test * Remove remaining assert_series_equal * Remove explicit cudf imports from many tests * Resolve rex test failures * Remove some additional compute calls * Consolidate sorting tests with getfixturevalue * Fix failed join test * Remove breakpoint * Use custom assert_eq function for tests * Resolve test failures / seg faults * Remove unnecessary testing utils * Resolve local test failures * Generalize RAND test * Avoid closing client if using independent cluster * Fix failures on Windows * Resolve black failures * Make random test variables more clear --- tests/integration/fixtures.py | 18 +- tests/integration/test_analyze.py | 16 +- tests/integration/test_compatibility.py | 8 +- tests/integration/test_complex.py | 7 +- tests/integration/test_create.py | 83 ++--- tests/integration/test_distributeby.py | 15 +- tests/integration/test_filter.py | 47 ++- tests/integration/test_fugue.py | 32 +- tests/integration/test_function.py | 66 ++-- tests/integration/test_groupby.py | 281 +++++++++-------- tests/integration/test_hive.py | 16 +- tests/integration/test_intake.py | 8 +- tests/integration/test_jdbc.py | 5 + tests/integration/test_join.py | 170 +++++------ tests/integration/test_model.py | 16 +- tests/integration/test_over.py | 78 ++--- tests/integration/test_rex.py | 134 ++++---- tests/integration/test_sample.py | 11 - tests/integration/test_schema.py | 14 +- tests/integration/test_select.py | 62 ++-- tests/integration/test_server.py | 5 +- tests/integration/test_show.py | 82 ++--- tests/integration/test_sort.py | 390 ++++++------------------ tests/integration/test_union.py | 17 +- tests/unit/test_call.py | 68 ++--- tests/unit/test_context.py | 17 +- tests/utils.py | 13 + 27 files changed, 651 insertions(+), 1028 deletions(-) create mode 100644 tests/utils.py diff --git a/tests/integration/fixtures.py b/tests/integration/fixtures.py index 5be6a930b..e1f53c6db 100644 --- a/tests/integration/fixtures.py +++ b/tests/integration/fixtures.py @@ -5,9 +5,9 @@ import numpy as np import pandas as pd import pytest -from dask.datasets import timeseries from dask.distributed import Client -from pandas.testing import assert_frame_equal + +from tests.utils import assert_eq try: import cudf @@ -23,18 +23,6 @@ SCHEDULER_ADDR = os.getenv("DASK_SQL_TEST_SCHEDULER", None) -@pytest.fixture() -def timeseries_df(c): - pdf = timeseries(freq="1d").compute().reset_index(drop=True) - # impute nans in pandas dataframe - col1_index = np.random.randint(0, 30, size=int(pdf.shape[0] * 0.2)) - col2_index = np.random.randint(0, 30, size=int(pdf.shape[0] * 0.3)) - pdf.loc[col1_index, "x"] = np.nan - pdf.loc[col2_index, "y"] = np.nan - c.create_table("timeseries", pdf, persist=True) - return pdf - - @pytest.fixture() def df_simple(): return pd.DataFrame({"a": [1, 2, 3], "b": [1.1, 2.2, 3.3]}) @@ -311,7 +299,7 @@ def _assert_query_gives_same_result(query, sort_columns=None, **kwargs): sql_result = sql_result.reset_index(drop=True) dask_result = dask_result.reset_index(drop=True) - assert_frame_equal(sql_result, dask_result, check_dtype=False, **kwargs) + assert_eq(sql_result, dask_result, check_dtype=False, **kwargs) return _assert_query_gives_same_result diff --git a/tests/integration/test_analyze.py b/tests/integration/test_analyze.py index cb3b70d2f..cd51ce1b8 100644 --- a/tests/integration/test_analyze.py +++ b/tests/integration/test_analyze.py @@ -1,11 +1,10 @@ -import numpy as np import pandas as pd -from pandas.testing import assert_frame_equal + +from tests.utils import assert_eq def test_analyze(c, df): result_df = c.sql("ANALYZE TABLE df COMPUTE STATISTICS FOR ALL COLUMNS") - result_df = result_df.compute() expected_df = pd.DataFrame( { @@ -15,8 +14,7 @@ def test_analyze(c, df): df.a.std(), 1.0, 2.0, - # That is actually wrong. But the approximate quantile function in dask gives a different result than the actual computation - result_df["a"].iloc[5], + 2.0, # incorrect, but what Dask gives for approx quantile 3.0, 3.0, "double", @@ -50,12 +48,8 @@ def test_analyze(c, df): ) # The percentiles are calculated only approximately, therefore we do not use exact matching - p = ["25%", "50%", "75%"] - result_df.loc[p, :] = result_df.loc[p, :].astype(float).apply(np.ceil) - expected_df.loc[p, :] = expected_df.loc[p, :].astype(float).apply(np.ceil) - assert_frame_equal(result_df, expected_df, check_exact=False) + assert_eq(result_df, expected_df, rtol=0.135) result_df = c.sql("ANALYZE TABLE df COMPUTE STATISTICS FOR COLUMNS a") - result_df = result_df.compute() - assert_frame_equal(result_df, expected_df[["a"]]) + assert_eq(result_df, expected_df[["a"]], rtol=0.135) diff --git a/tests/integration/test_compatibility.py b/tests/integration/test_compatibility.py index e087460df..63c1668b2 100644 --- a/tests/integration/test_compatibility.py +++ b/tests/integration/test_compatibility.py @@ -14,14 +14,14 @@ import numpy as np import pandas as pd -from pandas.testing import assert_frame_equal from dask_sql import Context from dask_sql.utils import ParsingException +from tests.utils import assert_eq def cast_datetime_to_string(df): - cols = df.select_dtypes(include=["datetime64[ns]"]).columns + cols = df.select_dtypes(include=["datetime64[ns]"]).columns.tolist() # Casting to object first as # directly converting to string looses second precision df[cols] = df[cols].astype("object").astype("string") @@ -36,7 +36,7 @@ def eq_sqlite(sql, **dfs): c.create_table(name, df) df.to_sql(name, engine, index=False) - dask_result = c.sql(sql).compute().reset_index(drop=True) + dask_result = c.sql(sql).reset_index(drop=True) sqlite_result = pd.read_sql(sql, engine).reset_index(drop=True) # casting to object to ensure equality with sql-lite @@ -47,7 +47,7 @@ def eq_sqlite(sql, **dfs): dask_result = dask_result.fillna(np.NaN) sqlite_result = sqlite_result.fillna(np.NaN) - assert_frame_equal(dask_result, sqlite_result, check_dtype=False) + assert_eq(dask_result, sqlite_result, check_dtype=False) def make_rand_df(size: int, **kwargs): diff --git a/tests/integration/test_complex.py b/tests/integration/test_complex.py index 0ebdd1a76..fc79f0a11 100644 --- a/tests/integration/test_complex.py +++ b/tests/integration/test_complex.py @@ -28,9 +28,6 @@ def test_complex_query(c): lhs.name = rhs.max_name AND lhs.x = rhs.max_x """ - ) + ).compute() - # should not fail - df = result.compute() - - assert len(df) > 0 + assert len(result) > 0 diff --git a/tests/integration/test_create.py b/tests/integration/test_create.py index cac4a3099..456435b7e 100644 --- a/tests/integration/test_create.py +++ b/tests/integration/test_create.py @@ -1,9 +1,9 @@ import dask.dataframe as dd import pandas as pd import pytest -from pandas.testing import assert_frame_equal import dask_sql +from tests.utils import assert_eq @pytest.mark.parametrize("gpu", [False, pytest.param(True, marks=pytest.mark.gpu)]) @@ -26,12 +26,9 @@ def test_create_from_csv(c, df, temporary_data_file, gpu): """ SELECT * FROM new_table """ - ).compute() - - if gpu: - result_df = result_df.to_pandas() + ) - assert_frame_equal(result_df, df) + assert_eq(result_df, df) @pytest.mark.parametrize( @@ -60,12 +57,9 @@ def test_cluster_memory(client, c, df, gpu): """ SELECT * FROM new_table """ - ).compute() - - if gpu: - return_df = return_df.to_pandas() + ) - assert_frame_equal(df, return_df) + assert_eq(df, return_df) client.unpublish_dataset("df") @@ -91,12 +85,9 @@ def test_create_from_csv_persist(c, df, temporary_data_file, gpu): """ SELECT * FROM new_table """ - ).compute() - - if gpu: - return_df = return_df.to_pandas() + ) - assert_frame_equal(df, return_df) + assert_eq(df, return_df) def test_wrong_create(c): @@ -139,9 +130,9 @@ def test_create_from_query(c, df): """ SELECT * FROM new_table """ - ).compute() + ) - assert_frame_equal(df, return_df) + assert_eq(df, return_df) c.sql( """ @@ -157,9 +148,9 @@ def test_create_from_query(c, df): """ SELECT * FROM new_table """ - ).compute() + ) - assert_frame_equal(df, return_df) + assert_eq(df, return_df) @pytest.mark.parametrize( @@ -210,27 +201,19 @@ def test_view_table_persist(c, temporary_data_file, df, gpu): """ ) - from_view = c.sql("SELECT c FROM count_view").compute() - from_table = c.sql("SELECT c FROM count_table").compute() - - if gpu: - from_view = from_view.to_pandas() - from_table = from_table.to_pandas() + from_view = c.sql("SELECT c FROM count_view") + from_table = c.sql("SELECT c FROM count_table") - assert_frame_equal(from_view, pd.DataFrame({"c": [700]})) - assert_frame_equal(from_table, pd.DataFrame({"c": [700]})) + assert_eq(from_view, pd.DataFrame({"c": [700]})) + assert_eq(from_table, pd.DataFrame({"c": [700]})) df.iloc[:10].to_csv(temporary_data_file, index=False) - from_view = c.sql("SELECT c FROM count_view").compute() - from_table = c.sql("SELECT c FROM count_table").compute() + from_view = c.sql("SELECT c FROM count_view") + from_table = c.sql("SELECT c FROM count_table") - if gpu: - from_view = from_view.to_pandas() - from_table = from_table.to_pandas() - - assert_frame_equal(from_view, pd.DataFrame({"c": [10]})) - assert_frame_equal(from_table, pd.DataFrame({"c": [700]})) + assert_eq(from_view, pd.DataFrame({"c": [10]})) + assert_eq(from_table, pd.DataFrame({"c": [700]})) def test_replace_and_error(c, temporary_data_file, df): @@ -244,8 +227,8 @@ def test_replace_and_error(c, temporary_data_file, df): """ ) - assert_frame_equal( - c.sql("SELECT a FROM new_table").compute(), + assert_eq( + c.sql("SELECT a FROM new_table"), pd.DataFrame({"a": [1]}), check_dtype=False, ) @@ -271,8 +254,8 @@ def test_replace_and_error(c, temporary_data_file, df): """ ) - assert_frame_equal( - c.sql("SELECT a FROM new_table").compute(), + assert_eq( + c.sql("SELECT a FROM new_table"), pd.DataFrame({"a": [1]}), check_dtype=False, ) @@ -287,8 +270,8 @@ def test_replace_and_error(c, temporary_data_file, df): """ ) - assert_frame_equal( - c.sql("SELECT a FROM new_table").compute(), + assert_eq( + c.sql("SELECT a FROM new_table"), pd.DataFrame({"a": [2]}), check_dtype=False, ) @@ -308,8 +291,8 @@ def test_replace_and_error(c, temporary_data_file, df): """ ) - assert_frame_equal( - c.sql("SELECT a FROM new_table").compute(), + assert_eq( + c.sql("SELECT a FROM new_table"), pd.DataFrame({"a": [3]}), check_dtype=False, ) @@ -338,8 +321,8 @@ def test_replace_and_error(c, temporary_data_file, df): """ ) - assert_frame_equal( - c.sql("SELECT a FROM new_table").compute(), + assert_eq( + c.sql("SELECT a FROM new_table"), pd.DataFrame({"a": [3]}), check_dtype=False, ) @@ -355,13 +338,9 @@ def test_replace_and_error(c, temporary_data_file, df): """ ) - result_df = c.sql( - """ - SELECT * FROM new_table - """ - ).compute() + result_df = c.sql("SELECT * FROM new_table") - assert_frame_equal(result_df, df) + assert_eq(result_df, df) def test_drop(c): diff --git a/tests/integration/test_distributeby.py b/tests/integration/test_distributeby.py index 93d3b3426..c865185ce 100644 --- a/tests/integration/test_distributeby.py +++ b/tests/integration/test_distributeby.py @@ -2,24 +2,13 @@ import pandas as pd import pytest -try: - import cudf -except ImportError: - cudf = None - @pytest.mark.parametrize("gpu", [False, pytest.param(True, marks=pytest.mark.gpu)]) def test_distribute_by(c, gpu): - - if gpu: - xd = cudf - else: - xd = pd - - df = xd.DataFrame({"id": [0, 1, 2, 1, 2, 3], "val": [0, 1, 2, 1, 2, 3]}) + df = pd.DataFrame({"id": [0, 1, 2, 1, 2, 3], "val": [0, 1, 2, 1, 2, 3]}) ddf = dd.from_pandas(df, npartitions=2) - c.create_table("test", ddf) + c.create_table("test", ddf, gpu=gpu) partitioned_ddf = c.sql( """ SELECT diff --git a/tests/integration/test_filter.py b/tests/integration/test_filter.py index a5231a5e2..192880267 100644 --- a/tests/integration/test_filter.py +++ b/tests/integration/test_filter.py @@ -2,51 +2,45 @@ import pandas as pd import pytest from dask.utils_test import hlg_layer -from pandas.testing import assert_frame_equal from dask_sql._compat import INT_NAN_IMPLEMENTED +from tests.utils import assert_eq def test_filter(c, df): return_df = c.sql("SELECT * FROM df WHERE a < 2") - return_df = return_df.compute() expected_df = df[df["a"] < 2] - assert_frame_equal(return_df, expected_df) + assert_eq(return_df, expected_df) def test_filter_scalar(c, df): return_df = c.sql("SELECT * FROM df WHERE True") - return_df = return_df.compute() expected_df = df - assert_frame_equal(return_df, expected_df) + assert_eq(return_df, expected_df) return_df = c.sql("SELECT * FROM df WHERE False") - return_df = return_df.compute() expected_df = df.head(0) - assert_frame_equal(return_df, expected_df, check_index_type=False) + assert_eq(return_df, expected_df, check_index_type=False) return_df = c.sql("SELECT * FROM df WHERE (1 = 1)") - return_df = return_df.compute() expected_df = df - assert_frame_equal(return_df, expected_df) + assert_eq(return_df, expected_df) return_df = c.sql("SELECT * FROM df WHERE (1 = 0)") - return_df = return_df.compute() expected_df = df.head(0) - assert_frame_equal(return_df, expected_df, check_index_type=False) + assert_eq(return_df, expected_df, check_index_type=False) def test_filter_complicated(c, df): return_df = c.sql("SELECT * FROM df WHERE a < 3 AND (b > 1 AND b < 3)") - return_df = return_df.compute() expected_df = df[((df["a"] < 3) & ((df["b"] > 1) & (df["b"] < 3)))] - assert_frame_equal( + assert_eq( return_df, expected_df, ) @@ -54,13 +48,12 @@ def test_filter_complicated(c, df): def test_filter_with_nan(c): return_df = c.sql("SELECT * FROM user_table_nan WHERE c = 3") - return_df = return_df.compute() if INT_NAN_IMPLEMENTED: expected_df = pd.DataFrame({"c": [3]}, dtype="Int8") else: expected_df = pd.DataFrame({"c": [3]}, dtype="float") - assert_frame_equal( + assert_eq( return_df, expected_df, ) @@ -68,9 +61,8 @@ def test_filter_with_nan(c): def test_string_filter(c, string_table): return_df = c.sql("SELECT * FROM string_table WHERE a = 'a normal string'") - return_df = return_df.compute() - assert_frame_equal( + assert_eq( return_df, string_table.head(1), ) @@ -96,7 +88,7 @@ def test_filter_cast_date(c, input_table, request): datetime_table["timezone"].astype(" pd.Timestamp("2014-08-01") ] - dd.assert_eq(return_df, expected_df) + assert_eq(return_df, expected_df) @pytest.mark.parametrize( @@ -119,19 +111,19 @@ def test_filter_cast_timestamp(c, input_table, request): datetime_table["timezone"].astype("= pd.Timestamp("2014-08-01 23:00:00") ] - dd.assert_eq(return_df, expected_df) + assert_eq(return_df, expected_df) def test_filter_year(c): df = pd.DataFrame({"year": [2015, 2016], "month": [2, 3], "day": [4, 5]}) - df["dt"] = pd.to_datetime(df) c.create_table("datetime_test", df) - actual_df = c.sql("select * from datetime_test where year(dt) < 2016").compute() + + return_df = c.sql("select * from datetime_test where year(dt) < 2016") expected_df = df[df["year"] < 2016] - assert_frame_equal(expected_df, actual_df) + assert_eq(expected_df, return_df) @pytest.mark.parametrize( @@ -199,9 +191,11 @@ def test_predicate_pushdown(c, parquet_ddf, query, df_func, filters): assert got_filters == expect_filters # Check computed result is correct - df = parquet_ddf.compute() + df = parquet_ddf expected_df = df_func(df) - dd.assert_eq(return_df, expected_df) + + # TODO: divisions should be consistent when successfully doing predicate pushdown + assert_eq(return_df, expected_df, check_divisions=False) def test_filtered_csv(tmpdir, c): @@ -229,6 +223,7 @@ def test_filtered_csv(tmpdir, c): c.drop_table("my_csv_table") # Check computed result is correct - df = csv_ddf.compute() + df = csv_ddf expected_df = df[df["b"] < 10] - dd.assert_eq(return_df, expected_df) + + assert_eq(return_df, expected_df) diff --git a/tests/integration/test_fugue.py b/tests/integration/test_fugue.py index ba3acd00c..951bf7a48 100644 --- a/tests/integration/test_fugue.py +++ b/tests/integration/test_fugue.py @@ -1,10 +1,10 @@ import dask.dataframe as dd import pandas as pd import pytest -from pandas.testing import assert_frame_equal from dask_sql import Context from tests.integration.fixtures import skip_if_external_scheduler +from tests.utils import assert_eq fugue_sql = pytest.importorskip("fugue_sql") @@ -14,44 +14,38 @@ def test_simple_statement(): - dag = fugue_sql.FugueSQLWorkflow() - df = dag.df([[0, "hello"], [1, "world"]], "a:int64,b:str") - dag( - """ - SELECT * FROM df WHERE a > 0 YIELD DATAFRAME AS result - """ - ) - result = dag.run(DaskSQLExecutionEngine) + with fugue_sql.FugueSQLWorkflow(DaskSQLExecutionEngine) as dag: + df = dag.df([[0, "hello"], [1, "world"]], "a:int64,b:str") + dag("SELECT * FROM df WHERE a > 0 YIELD DATAFRAME AS result") + result = dag.run() return_df = result["result"].as_pandas() - assert_frame_equal(return_df, pd.DataFrame({"a": [1], "b": ["world"]})) + assert_eq(return_df, pd.DataFrame({"a": [1], "b": ["world"]})) # A more elegant way to do things pdf = pd.DataFrame([[0, "hello"], [1, "world"]], columns=["a", "b"]) result = fugue_sql.fsql( - """ - SELECT * FROM df WHERE a > 0 YIELD DATAFRAME AS result - """, + "SELECT * FROM df WHERE a > 0 YIELD DATAFRAME AS result", df=pdf, ).run("dask") return_df = result["result"].as_pandas() - assert_frame_equal(return_df, pd.DataFrame({"a": [1], "b": ["world"]})) + assert_eq(return_df, pd.DataFrame({"a": [1], "b": ["world"]})) # TODO: Revisit fixing this on an independant cluster (without dask-sql) based on the # discussion in https://github.com/dask-contrib/dask-sql/issues/407 @skip_if_external_scheduler def test_fsql(): - def assert_eq(df: pd.DataFrame) -> None: - assert_frame_equal(df, pd.DataFrame({"a": [1]})) + def assert_fsql(df: pd.DataFrame) -> None: + assert_eq(df, pd.DataFrame({"a": [1]})) # the simplest case: the SQL does not use any input and does not generate output fsql_dask( """ CREATE [[0],[1]] SCHEMA a:long SELECT * WHERE a>0 - OUTPUT USING assert_eq + OUTPUT USING assert_fsql """ ) @@ -64,7 +58,7 @@ def assert_eq(df: pd.DataFrame) -> None: fsql_dask( """ SELECT * FROM df WHERE a>0 - OUTPUT USING assert_eq + OUTPUT USING assert_fsql """, c, ) @@ -74,7 +68,7 @@ def assert_eq(df: pd.DataFrame) -> None: result = fsql_dask( """ x=SELECT * FROM df WHERE a>0 - OUTPUT USING assert_eq + OUTPUT USING assert_fsql """, c, register=True, diff --git a/tests/integration/test_function.py b/tests/integration/test_function.py index c6342877c..92fc58b14 100644 --- a/tests/integration/test_function.py +++ b/tests/integration/test_function.py @@ -4,7 +4,8 @@ import dask.dataframe as dd import numpy as np import pytest -from pandas.testing import assert_frame_equal + +from tests.utils import assert_eq def test_custom_function(c, df): @@ -13,15 +14,9 @@ def f(x): c.register_function(f, "f", [("x", np.float64)], np.float64) - return_df = c.sql( - """ - SELECT F(a) AS a - FROM df - """ - ) - return_df = return_df.compute() + return_df = c.sql("SELECT F(a) AS a FROM df") - assert_frame_equal(return_df.reset_index(drop=True), df[["a"]] ** 2) + assert_eq(return_df, df[["a"]] ** 2) def test_custom_function_row(c, df): @@ -30,15 +25,9 @@ def f(row): c.register_function(f, "f", [("x", np.float64)], np.float64, row_udf=True) - return_df = c.sql( - """ - SELECT F(a) AS a - FROM df - """ - ) - return_df = return_df.compute() + return_df = c.sql("SELECT F(a) AS a FROM df") - assert_frame_equal(return_df.reset_index(drop=True), df[["a"]] ** 2) + assert_eq(return_df, df[["a"]] ** 2) @pytest.mark.parametrize("colnames", list(itertools.combinations(["a", "b", "c"], 2))) @@ -58,7 +47,7 @@ def f(row): expect = df_wide[colname_x] + df_wide[colname_y] got = return_df.iloc[:, 0] - dd.assert_eq(expect, got, check_names=False) + assert_eq(expect, got, check_names=False) @pytest.mark.parametrize( @@ -75,15 +64,10 @@ def f(row): return c.register_function(f, "f", [("x", np.float64)], retty, row_udf=True) - return_df = c.sql( - """ - SELECT F(a) AS a - FROM df - """ - ) - return_df = return_df.compute() - expectation = (df[["a"]] ** 2).astype(retty) - assert_frame_equal(return_df.reset_index(drop=True), expectation) + + return_df = c.sql("SELECT F(a) AS a FROM df") + + assert_eq(return_df, (df[["a"]] ** 2).astype(retty)) # Test row UDFs with one arg @@ -102,12 +86,10 @@ def f(row, k): f, "f", [("a", np.int64), ("k", const_type)], retty, row_udf=True ) - statement = f"SELECT F(a, {k}) as a from df" + return_df = c.sql(f"SELECT F(a, {k}) as a from df") + expected_df = op(df[["a"]], k).astype(retty) - return_df = c.sql(statement) - return_df = return_df.compute() - expectation = op(df[["a"]], k).astype(retty) - assert_frame_equal(return_df.reset_index(drop=True), expectation) + assert_eq(return_df, expected_df) # Test row UDFs with two args @@ -135,13 +117,10 @@ def f(row, k1, k2): row_udf=True, ) - statement = f"SELECT F(a, {k1}, {k2}) as a from df" - - return_df = c.sql(statement) - return_df = return_df.compute() + return_df = c.sql(f"SELECT F(a, {k1}, {k2}) as a from df") + expected_df = op(op(df[["a"]], k1), k2).astype(retty) - expectation = op(op(df[["a"]], k1), k2).astype(retty) - assert_frame_equal(return_df.reset_index(drop=True), expectation) + assert_eq(return_df, expected_df) def test_multiple_definitions(c, df_simple): @@ -157,9 +136,9 @@ def f(x): FROM df_simple """ ) - return_df = return_df.compute() + expected_df = df_simple[["a", "b"]] ** 2 - assert_frame_equal(return_df.reset_index(drop=True), df_simple[["a", "b"]] ** 2) + assert_eq(return_df, expected_df) def f(x): return x**3 @@ -173,9 +152,9 @@ def f(x): FROM df_simple """ ) - return_df = return_df.compute() + expected_df = df_simple[["a", "b"]] ** 3 - assert_frame_equal(return_df.reset_index(drop=True), df_simple[["a", "b"]] ** 3) + assert_eq(return_df, expected_df) def test_aggregate_function(c): @@ -188,9 +167,8 @@ def test_aggregate_function(c): FROM df """ ) - return_df = return_df.compute() - assert (return_df["test"] == return_df["S"]).all() + assert_eq(return_df["test"], return_df["S"], check_names=False) def test_reregistration(c): diff --git a/tests/integration/test_groupby.py b/tests/integration/test_groupby.py index 0fef45679..658ad4fa2 100644 --- a/tests/integration/test_groupby.py +++ b/tests/integration/test_groupby.py @@ -1,12 +1,29 @@ +import dask.dataframe as dd import numpy as np import pandas as pd import pytest -from dask import dataframe as dd -from pandas.testing import assert_frame_equal, assert_series_equal +from dask.datasets import timeseries + +from tests.utils import assert_eq + + +@pytest.fixture() +def timeseries_df(c): + pdf = timeseries(freq="1d").compute().reset_index(drop=True) + + # input nans in pandas dataframe + col1_index = np.random.randint(0, 30, size=int(pdf.shape[0] * 0.2)) + col2_index = np.random.randint(0, 30, size=int(pdf.shape[0] * 0.3)) + pdf.loc[col1_index, "x"] = np.nan + pdf.loc[col2_index, "y"] = np.nan + + c.create_table("timeseries", pdf, persist=True) + + return None def test_group_by(c): - df = c.sql( + return_df = c.sql( """ SELECT user_id, SUM(b) AS "S" @@ -14,10 +31,9 @@ def test_group_by(c): GROUP BY user_id """ ) - df = df.compute() - expected_df = pd.DataFrame({"user_id": [1, 2, 3], "S": [3, 4, 3]}) - assert_frame_equal(df.sort_values("user_id").reset_index(drop=True), expected_df) + + assert_eq(return_df.sort_values("user_id").reset_index(drop=True), expected_df) def test_group_by_all(c, df): @@ -28,12 +44,11 @@ def test_group_by_all(c, df): FROM user_table_1 """ ) - result_df = result_df.compute() - expected_df = pd.DataFrame({"S": [10], "X": [8]}) expected_df["S"] = expected_df["S"].astype("int64") expected_df["X"] = expected_df["X"].astype("int32") - assert_frame_equal(result_df, expected_df) + + assert_eq(result_df, expected_df) result_df = c.sql( """ @@ -48,8 +63,6 @@ def test_group_by_all(c, df): FROM df """ ) - result_df = result_df.compute() - expected_df = pd.DataFrame( { "sum_a": [df.a.sum()], @@ -61,11 +74,12 @@ def test_group_by_all(c, df): "mix_3": [(df.a + df.b).mean()], } ) - assert_frame_equal(result_df, expected_df) + + assert_eq(result_df, expected_df) def test_group_by_filtered(c): - df = c.sql( + return_df = c.sql( """ SELECT SUM(b) FILTER (WHERE user_id = 2) AS "S1", @@ -73,14 +87,11 @@ def test_group_by_filtered(c): FROM user_table_1 """ ) - df = df.compute() - expected_df = pd.DataFrame({"S1": [4], "S2": [10]}, dtype="int64") - assert_frame_equal(df, expected_df) + assert_eq(return_df, expected_df) -def test_group_by_filtered2(c): - df = c.sql( + return_df = c.sql( """ SELECT user_id, @@ -90,8 +101,6 @@ def test_group_by_filtered2(c): GROUP BY user_id """ ) - df = df.compute() - expected_df = pd.DataFrame( { "user_id": [1, 2, 3], @@ -99,23 +108,22 @@ def test_group_by_filtered2(c): "S2": [3, 4, 3], }, ) - assert_frame_equal(df, expected_df) - df = c.sql( + assert_eq(return_df, expected_df) + + return_df = c.sql( """ SELECT SUM(b) FILTER (WHERE user_id = 2) AS "S1" FROM user_table_1 """ ) - df = df.compute() - expected_df = pd.DataFrame({"S1": [4]}) - assert_frame_equal(df, expected_df) + assert_eq(return_df, expected_df) def test_group_by_case(c): - df = c.sql( + return_df = c.sql( """ SELECT user_id + 1 AS "A", SUM(CASE WHEN b = 3 THEN 1 END) AS "S" @@ -123,17 +131,18 @@ def test_group_by_case(c): GROUP BY user_id + 1 """ ) - df = df.compute() - expected_df = pd.DataFrame({"A": [2, 3, 4], "S": [1, 1, 1]}) + # Do not check dtypes, as pandas versions are inconsistent here - assert_frame_equal( - df.sort_values("A").reset_index(drop=True), expected_df, check_dtype=False + assert_eq( + return_df.sort_values("A").reset_index(drop=True), + expected_df, + check_dtype=False, ) def test_group_by_nan(c): - df = c.sql( + return_df = c.sql( """ SELECT c @@ -141,18 +150,17 @@ def test_group_by_nan(c): GROUP BY c """ ) - df = df.compute() - expected_df = pd.DataFrame({"c": [3, float("nan"), 1]}) + # The dtype in pandas 1.0.5 and pandas 1.1.0 are different, so - # we can not check here - assert_frame_equal( - df.sort_values("c").reset_index(drop=True), + # we cannot check here + assert_eq( + return_df.sort_values("c").reset_index(drop=True), expected_df.sort_values("c").reset_index(drop=True), check_dtype=False, ) - df = c.sql( + return_df = c.sql( """ SELECT c @@ -160,18 +168,17 @@ def test_group_by_nan(c): GROUP BY c """ ) - df = df.compute() - expected_df = pd.DataFrame({"c": [3, 1, float("inf")]}) expected_df["c"] = expected_df["c"].astype("float64") - assert_frame_equal( - df.sort_values("c").reset_index(drop=True), + + assert_eq( + return_df.sort_values("c").reset_index(drop=True), expected_df.sort_values("c").reset_index(drop=True), ) def test_aggregations(c): - df = c.sql( + return_df = c.sql( """ SELECT user_id, @@ -185,8 +192,6 @@ def test_aggregations(c): GROUP BY user_id """ ) - df = df.compute() - expected_df = pd.DataFrame( { "user_id": [1, 2, 3], @@ -199,9 +204,10 @@ def test_aggregations(c): } ) expected_df["a"] = expected_df["a"].astype("float64") - assert_frame_equal(df.sort_values("user_id").reset_index(drop=True), expected_df) - df = c.sql( + assert_eq(return_df.sort_values("user_id").reset_index(drop=True), expected_df) + + return_df = c.sql( """ SELECT user_id, @@ -215,7 +221,6 @@ def test_aggregations(c): GROUP BY user_id """ ) - df = df.compute() expected_df = pd.DataFrame( { @@ -228,9 +233,9 @@ def test_aggregations(c): "a": [1.5, 3, 4], } ) - assert_frame_equal(df.sort_values("user_id").reset_index(drop=True), expected_df) + assert_eq(return_df.sort_values("user_id").reset_index(drop=True), expected_df) - df = c.sql( + return_df = c.sql( """ SELECT MAX(a) AS "max", @@ -238,27 +243,25 @@ def test_aggregations(c): FROM string_table """ ) - df = df.compute() - expected_df = pd.DataFrame({"max": ["a normal string"], "min": ["%_%"]}) - assert_frame_equal(df.reset_index(drop=True), expected_df) + assert_eq(return_df.reset_index(drop=True), expected_df) -def test_stats_aggregation(c, timeseries_df): - # # test regr_count - regr_count = ( - c.sql( - """ - SELECT name, count(x) filter (where y is not null) as expected, - regr_count(y, x) as calculated from timeseries group by name +def test_stats_aggregation(c, timeseries_df): + # test regr_count + regr_count = c.sql( + """ + SELECT + name, + COUNT(x) FILTER (WHERE y IS NOT NULL) AS expected, + REGR_COUNT(y, x) AS calculated + FROM timeseries + GROUP BY name """ - ) - .compute() - .fillna(0) - ) + ).fillna(0) - assert_series_equal( + assert_eq( regr_count["expected"], regr_count["calculated"], check_dtype=False, @@ -266,17 +269,19 @@ def test_stats_aggregation(c, timeseries_df): ) # test regr_syy - regr_syy = ( - c.sql( - """ - SELECT name, (regr_count(y, x)*var_pop(y)) as expected, regr_syy(y, x) as calculated - FROM timeseries WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY name + regr_syy = c.sql( + """ + SELECT + name, + (REGR_COUNT(y, x) * VAR_POP(y)) AS expected, + REGR_SYY(y, x) AS calculated + FROM timeseries + WHERE x IS NOT NULL AND y IS NOT NULL + GROUP BY name """ - ) - .compute() - .fillna(0) - ) - assert_series_equal( + ).fillna(0) + + assert_eq( regr_syy["expected"], regr_syy["calculated"], check_dtype=False, @@ -284,17 +289,19 @@ def test_stats_aggregation(c, timeseries_df): ) # test regr_sxx - regr_sxx = ( - c.sql( - """ - SELECT name,(regr_count(y, x)*var_pop(x)) as expected, regr_sxx(y,x) as calculated - FROM timeseries WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY name + regr_sxx = c.sql( + """ + SELECT + name, + (REGR_COUNT(y, x) * VAR_POP(x)) AS expected, + REGR_SXX(y,x) AS calculated + FROM timeseries + WHERE x IS NOT NULL AND y IS NOT NULL + GROUP BY name """ - ) - .compute() - .fillna(0) - ) - assert_series_equal( + ).fillna(0) + + assert_eq( regr_sxx["expected"], regr_sxx["calculated"], check_dtype=False, @@ -302,24 +309,26 @@ def test_stats_aggregation(c, timeseries_df): ) # test covar_pop - covar_pop = ( - c.sql( - """ - with temp_agg as ( - select name,avg(y) filter (where x is not null) as avg_y, - avg(x) filter (where y is not null) as avg_x - from timeseries group by name - ) - select ts.name,sum((y - avg_y) * (x - avg_x)) /regr_count(y, x) as expected, - covar_pop(y,x) as calculated from timeseries as ts - join temp_agg as ta on ts.name =ta.name - group by ts.name + covar_pop = c.sql( + """ + WITH temp_agg AS ( + SELECT + name, + AVG(y) FILTER (WHERE x IS NOT NULL) as avg_y, + AVG(x) FILTER (WHERE x IS NOT NULL) as avg_x + FROM timeseries + GROUP BY name + ) SELECT + ts.name, + SUM((y - avg_y) * (x - avg_x)) / REGR_COUNT(y, x) AS expected, + COVAR_POP(y,x) AS calculated + FROM timeseries AS ts + JOIN temp_agg AS ta ON ts.name = ta.name + GROUP BY ts.name """ - ) - .compute() - .fillna(0) - ) - assert_series_equal( + ).fillna(0) + + assert_eq( covar_pop["expected"], covar_pop["calculated"], check_dtype=False, @@ -327,25 +336,26 @@ def test_stats_aggregation(c, timeseries_df): ) # test covar_samp - covar_samp = ( - c.sql( - """ - with temp_agg as ( - select name,avg(y) filter (where x is not null) as avg_y, - avg(x) filter (where y is not null) as avg_x - from timeseries group by name - ) - - select ts.name,sum((y - avg_y) * (x - avg_x)) /(regr_count(y, x)-1) as expected, - covar_samp(y,x) as calculated from timeseries as ts - join temp_agg as ta on ts.name =ta.name - group by ts.name + covar_samp = c.sql( + """ + WITH temp_agg AS ( + SELECT + name, + AVG(y) FILTER (WHERE x IS NOT NULL) as avg_y, + AVG(x) FILTER (WHERE x IS NOT NULL) as avg_x + FROM timeseries + GROUP BY name + ) SELECT + ts.name, + SUM((y - avg_y) * (x - avg_x)) / (REGR_COUNT(y, x) - 1) as expected, + COVAR_SAMP(y,x) AS calculated + FROM timeseries AS ts + JOIN temp_agg AS ta ON ts.name = ta.name + GROUP BY ts.name """ - ) - .compute() - .fillna(0) - ) - assert_series_equal( + ).fillna(0) + + assert_eq( covar_samp["expected"], covar_samp["calculated"], check_dtype=False, @@ -363,7 +373,8 @@ def test_stats_aggregation(c, timeseries_df): @pytest.mark.parametrize("split_out", [None, 2, 4]) def test_groupby_split_out(c, input_table, split_out, request): user_table = request.getfixturevalue(input_table) - df = c.sql( + + return_df = c.sql( f""" SELECT user_id, SUM(b) AS "S" @@ -373,12 +384,15 @@ def test_groupby_split_out(c, input_table, split_out, request): config_options={"sql.groupby.split_out": split_out}, ) expected_df = ( - user_table.groupby(by="user_id").agg({"b": "sum"}).reset_index(drop=False) + user_table.groupby(by="user_id") + .agg({"b": "sum"}) + .reset_index(drop=False) + .rename(columns={"b": "S"}) + .sort_values("user_id") ) - expected_df = expected_df.rename(columns={"b": "S"}) - expected_df = expected_df.sort_values("user_id") - assert df.npartitions == split_out if split_out else 1 - dd.assert_eq(df.compute().sort_values("user_id"), expected_df, check_index=False) + + assert return_df.npartitions == split_out if split_out else 1 + assert_eq(return_df.sort_values("user_id"), expected_df, check_index=False) @pytest.mark.parametrize( @@ -387,21 +401,20 @@ def test_groupby_split_out(c, input_table, split_out, request): (False, 2, 74), (False, 3, 68), (False, 4, 64), - pytest.param(True, 2, 91, marks=pytest.mark.gpu), - pytest.param(True, 3, 85, marks=pytest.mark.gpu), - pytest.param(True, 4, 81, marks=pytest.mark.gpu), + pytest.param(True, 2, 107, marks=pytest.mark.gpu), + pytest.param(True, 3, 101, marks=pytest.mark.gpu), + pytest.param(True, 4, 97, marks=pytest.mark.gpu), ], ) def test_groupby_split_every(c, gpu, split_every, expected_keys): - xd = pytest.importorskip("cudf") if gpu else pd input_ddf = dd.from_pandas( - xd.DataFrame({"user_id": [1, 2, 3, 4] * 16, "b": [5, 6, 7, 8] * 16}), + pd.DataFrame({"user_id": [1, 2, 3, 4] * 16, "b": [5, 6, 7, 8] * 16}), npartitions=16, ) # Need an input with multiple partitions to demonstrate split_every - c.create_table("split_every_input", input_ddf) + c.create_table("split_every_input", input_ddf, gpu=gpu) - df = c.sql( + return_df = c.sql( """ SELECT user_id, SUM(b) AS "S" @@ -418,7 +431,7 @@ def test_groupby_split_every(c, gpu, split_every, expected_keys): .sort_values("user_id") ) - assert len(df.dask.keys()) == expected_keys - dd.assert_eq(df, expected_df, check_index=False) + assert len(return_df.dask.keys()) == expected_keys + assert_eq(return_df, expected_df, check_index=False) c.drop_table("split_every_input") diff --git a/tests/integration/test_hive.py b/tests/integration/test_hive.py index 330888618..04e411f8d 100644 --- a/tests/integration/test_hive.py +++ b/tests/integration/test_hive.py @@ -5,9 +5,9 @@ import pandas as pd import pytest -from pandas.testing import assert_frame_equal from dask_sql.context import Context +from tests.utils import assert_eq pytestmark = pytest.mark.skipif( sys.platform == "win32", reason="hive testing not supported on Windows" @@ -181,18 +181,18 @@ def test_select(hive_cursor): c = Context() c.create_table("df", hive_cursor) - result_df = c.sql("SELECT * FROM df").compute().reset_index(drop=True) - df = pd.DataFrame({"i": [1, 2], "j": [2, 4]}).astype("int32") + result_df = c.sql("SELECT * FROM df") + expected_df = pd.DataFrame({"i": [1, 2], "j": [2, 4]}).astype("int32") - assert_frame_equal(df, result_df) + assert_eq(result_df, expected_df, check_index=False) def test_select_partitions(hive_cursor): c = Context() c.create_table("df_part", hive_cursor) - result_df = c.sql("SELECT * FROM df_part").compute().reset_index(drop=True) - df = pd.DataFrame({"i": [1, 2], "j": [2, 4]}).astype("int32") - df["j"] = df["j"].astype("int64") + result_df = c.sql("SELECT * FROM df_part") + expected_df = pd.DataFrame({"i": [1, 2], "j": [2, 4]}).astype("int32") + expected_df["j"] = expected_df["j"].astype("int64") - assert_frame_equal(df, result_df) + assert_eq(result_df, expected_df, check_index=False) diff --git a/tests/integration/test_intake.py b/tests/integration/test_intake.py index 4796b44d5..ea7937029 100644 --- a/tests/integration/test_intake.py +++ b/tests/integration/test_intake.py @@ -4,9 +4,9 @@ import pandas as pd import pytest -from pandas.testing import assert_frame_equal from dask_sql.context import Context +from tests.utils import assert_eq # skip the test if intake is not installed intake = pytest.importorskip("intake") @@ -40,10 +40,10 @@ def intake_catalog_location(): def check_read_table(c): - result_df = c.sql("SELECT * FROM df").compute().reset_index(drop=True) - df = pd.DataFrame({"a": [1], "b": [1.5]}) + result_df = c.sql("SELECT * FROM df").reset_index(drop=True) + expected_df = pd.DataFrame({"a": [1], "b": [1.5]}) - assert_frame_equal(df, result_df) + assert_eq(result_df, expected_df) def test_intake_catalog(intake_catalog_location): diff --git a/tests/integration/test_jdbc.py b/tests/integration/test_jdbc.py index f8426ae46..ce216da84 100644 --- a/tests/integration/test_jdbc.py +++ b/tests/integration/test_jdbc.py @@ -1,3 +1,4 @@ +import os from time import sleep import pandas as pd @@ -37,6 +38,10 @@ def app_client(c): yield TestClient(app) + # don't disconnect the client if using an independent cluster + if os.getenv("DASK_SQL_TEST_SCHEDULER", None) is None: + app.client.close() + def test_jdbc_has_schema(app_client, c): create_meta_data(c) diff --git a/tests/integration/test_join.py b/tests/integration/test_join.py index 44cd1e070..97c35e166 100644 --- a/tests/integration/test_join.py +++ b/tests/integration/test_join.py @@ -1,48 +1,52 @@ import dask.dataframe as dd import numpy as np import pandas as pd -from dask.dataframe.utils import assert_eq -from pandas.testing import assert_frame_equal from dask_sql import Context +from tests.utils import assert_eq def test_join(c): - df = c.sql( - "SELECT lhs.user_id, lhs.b, rhs.c FROM user_table_1 AS lhs JOIN user_table_2 AS rhs ON lhs.user_id = rhs.user_id" + return_df = c.sql( + """ + SELECT lhs.user_id, lhs.b, rhs.c + FROM user_table_1 AS lhs + JOIN user_table_2 AS rhs + ON lhs.user_id = rhs.user_id + """ ) - df = df.compute() - expected_df = pd.DataFrame( {"user_id": [1, 1, 2, 2], "b": [3, 3, 1, 3], "c": [1, 2, 3, 3]} ) - assert_frame_equal( - df.sort_values(["user_id", "b", "c"]).reset_index(drop=True), - expected_df, - ) + + assert_eq(return_df, expected_df, check_index=False) def test_join_inner(c): - df = c.sql( - "SELECT lhs.user_id, lhs.b, rhs.c FROM user_table_1 AS lhs INNER JOIN user_table_2 AS rhs ON lhs.user_id = rhs.user_id" + return_df = c.sql( + """ + SELECT lhs.user_id, lhs.b, rhs.c + FROM user_table_1 AS lhs + INNER JOIN user_table_2 AS rhs + ON lhs.user_id = rhs.user_id + """ ) - df = df.compute() - expected_df = pd.DataFrame( {"user_id": [1, 1, 2, 2], "b": [3, 3, 1, 3], "c": [1, 2, 3, 3]} ) - assert_frame_equal( - df.sort_values(["user_id", "b", "c"]).reset_index(drop=True), - expected_df, - ) + + assert_eq(return_df, expected_df, check_index=False) def test_join_outer(c): - df = c.sql( - "SELECT lhs.user_id, lhs.b, rhs.c FROM user_table_1 AS lhs FULL JOIN user_table_2 AS rhs ON lhs.user_id = rhs.user_id" + return_df = c.sql( + """ + SELECT lhs.user_id, lhs.b, rhs.c + FROM user_table_1 AS lhs + FULL JOIN user_table_2 AS rhs + ON lhs.user_id = rhs.user_id + """ ) - df = df.compute() - expected_df = pd.DataFrame( { # That is strange. Unfortunately, it seems dask fills in the @@ -52,17 +56,19 @@ def test_join_outer(c): "c": [1, 2, 3, 3, np.NaN, 4], } ) - assert_frame_equal( - df.sort_values(["user_id", "b", "c"]).reset_index(drop=True), expected_df - ) + + assert_eq(return_df, expected_df, check_index=False) def test_join_left(c): - df = c.sql( - "SELECT lhs.user_id, lhs.b, rhs.c FROM user_table_1 AS lhs LEFT JOIN user_table_2 AS rhs ON lhs.user_id = rhs.user_id" + return_df = c.sql( + """ + SELECT lhs.user_id, lhs.b, rhs.c + FROM user_table_1 AS lhs + LEFT JOIN user_table_2 AS rhs + ON lhs.user_id = rhs.user_id + """ ) - df = df.compute() - expected_df = pd.DataFrame( { # That is strange. Unfortunately, it seems dask fills in the @@ -72,18 +78,19 @@ def test_join_left(c): "c": [1, 2, 3, 3, np.NaN], } ) - assert_frame_equal( - df.sort_values(["user_id", "b", "c"]).reset_index(drop=True), - expected_df, - ) + + assert_eq(return_df, expected_df, check_index=False) def test_join_right(c): - df = c.sql( - "SELECT lhs.user_id, lhs.b, rhs.c FROM user_table_1 AS lhs RIGHT JOIN user_table_2 AS rhs ON lhs.user_id = rhs.user_id" + return_df = c.sql( + """ + SELECT lhs.user_id, lhs.b, rhs.c + FROM user_table_1 AS lhs + RIGHT JOIN user_table_2 AS rhs + ON lhs.user_id = rhs.user_id + """ ) - df = df.compute() - expected_df = pd.DataFrame( { # That is strange. Unfortunately, it seems dask fills in the @@ -93,36 +100,34 @@ def test_join_right(c): "c": [1, 2, 3, 3, 4], } ) - assert_frame_equal( - df.sort_values(["user_id", "b", "c"]).reset_index(drop=True), - expected_df, - ) + + assert_eq(return_df, expected_df, check_index=False) def test_join_complex(c): - df = c.sql( - "SELECT lhs.a, rhs.b FROM df_simple AS lhs JOIN df_simple AS rhs ON lhs.a < rhs.b", + return_df = c.sql( + """ + SELECT lhs.a, rhs.b + FROM df_simple AS lhs + JOIN df_simple AS rhs + ON lhs.a < rhs.b + """ ) - df = df.compute() - - df_expected = pd.DataFrame( + expected_df = pd.DataFrame( {"a": [1, 1, 1, 2, 2, 3], "b": [1.1, 2.2, 3.3, 2.2, 3.3, 3.3]} ) - assert_frame_equal(df.sort_values(["a", "b"]).reset_index(drop=True), df_expected) + assert_eq(return_df, expected_df, check_index=False) - df = c.sql( - """ - SELECT lhs.a, lhs.b, rhs.a, rhs.b - FROM - df_simple AS lhs - JOIN df_simple AS rhs - ON lhs.a < rhs.b AND lhs.b < rhs.a + return_df = c.sql( """ + SELECT lhs.a, lhs.b, rhs.a, rhs.b + FROM df_simple AS lhs + JOIN df_simple AS rhs + ON lhs.a < rhs.b AND lhs.b < rhs.a + """ ) - df = df.compute() - - df_expected = pd.DataFrame( + expected_df = pd.DataFrame( { "a": [1, 1, 2], "b": [1.1, 1.1, 2.2], @@ -131,43 +136,33 @@ def test_join_complex(c): } ) - assert_frame_equal(df.sort_values(["a", "b0"]).reset_index(drop=True), df_expected) - + assert_eq(return_df, expected_df, check_index=False) -def test_join_complex_2(c): - df = c.sql( + return_df = c.sql( """ - SELECT - lhs.user_id, lhs.b, rhs.user_id, rhs.c + SELECT lhs.user_id, lhs.b, rhs.user_id, rhs.c FROM user_table_1 AS lhs JOIN user_table_2 AS rhs - ON rhs.user_id = lhs.user_id AND rhs.c - lhs.b >= 0 + ON rhs.user_id = lhs.user_id AND rhs.c - lhs.b >= 0 """ ) - - df = df.compute() - - df_expected = pd.DataFrame( + expected_df = pd.DataFrame( {"user_id": [2, 2], "b": [1, 3], "user_id0": [2, 2], "c": [3, 3]} ) - assert_frame_equal(df.sort_values("b").reset_index(drop=True), df_expected) + assert_eq(return_df, expected_df, check_index=False) def test_join_literal(c): - df = c.sql( + return_df = c.sql( """ - SELECT - lhs.user_id, lhs.b, rhs.user_id, rhs.c + SELECT lhs.user_id, lhs.b, rhs.user_id, rhs.c FROM user_table_1 AS lhs JOIN user_table_2 AS rhs - ON True + ON True """ ) - - df = df.compute() - - df_expected = pd.DataFrame( + expected_df = pd.DataFrame( { "user_id": [2, 2, 2, 2, 1, 1, 1, 1, 2, 2, 2, 2, 3, 3, 3, 3], "b": [1, 1, 1, 1, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3], @@ -176,26 +171,19 @@ def test_join_literal(c): } ) - assert_frame_equal( - df.sort_values(["b", "user_id", "user_id0"]).reset_index(drop=True), - df_expected, - ) + assert_eq(return_df, expected_df, check_index=False) - df = c.sql( + return_df = c.sql( """ - SELECT - lhs.user_id, lhs.b, rhs.user_id, rhs.c + SELECT lhs.user_id, lhs.b, rhs.user_id, rhs.c FROM user_table_1 AS lhs JOIN user_table_2 AS rhs - ON False + ON False """ ) + expected_df = pd.DataFrame({"user_id": [], "b": [], "user_id0": [], "c": []}) - df = df.compute() - - df_expected = pd.DataFrame({"user_id": [], "b": [], "user_id0": [], "c": []}) - - assert_frame_equal(df.reset_index(), df_expected.reset_index(), check_dtype=False) + assert_eq(return_df, expected_df, check_dtype=False, check_index=False) def test_conditional_join(c): @@ -256,7 +244,7 @@ def test_join_case_projection_subquery(): c.create_table("sales", sales, persist=False) c.create_table("t_dim", t_dim, persist=False) - actual_df = c.sql( + c.sql( """ SELECT CASE WHEN pmc > 0.0 THEN CAST (amc AS DOUBLE) / CAST (pmc AS DOUBLE) ELSE -1.0 END AS am_pm_ratio FROM diff --git a/tests/integration/test_model.py b/tests/integration/test_model.py index 791ac0722..396d80573 100644 --- a/tests/integration/test_model.py +++ b/tests/integration/test_model.py @@ -8,6 +8,7 @@ from dask.datasets import timeseries from tests.integration.fixtures import skip_if_external_scheduler +from tests.utils import assert_eq try: import cuml @@ -231,10 +232,10 @@ def test_show_models(c, training_df): ) """ ) + result = c.sql("SHOW MODELS") expected = pd.DataFrame(["my_model1", "my_model2", "my_model3"], columns=["Models"]) - result: pd.DataFrame = c.sql("SHOW MODELS").compute() - # test - pd.testing.assert_frame_equal(expected, result) + + assert_eq(result, expected) def test_wrong_training_or_prediction(c, training_df): @@ -440,12 +441,9 @@ def test_describe_model(c, training_df): .sort_index() ) # test - result = ( - c.sql("DESCRIBE MODEL ex_describe_model") - .compute()["Params"] - .apply(lambda x: str(x)) - ) - pd.testing.assert_series_equal(expected_series, result) + result = c.sql("DESCRIBE MODEL ex_describe_model")["Params"].apply(lambda x: str(x)) + + assert_eq(expected_series, result) with pytest.raises(RuntimeError): c.sql("DESCRIBE MODEL undefined_model") diff --git a/tests/integration/test_over.py b/tests/integration/test_over.py index 20fa5b72c..f2ec4b9df 100644 --- a/tests/integration/test_over.py +++ b/tests/integration/test_over.py @@ -1,17 +1,10 @@ import pandas as pd -from pandas.testing import assert_frame_equal - -def assert_frame_equal_after_sorting(df, expected_df, columns=None, **kwargs): - columns = columns or ["user_id", "b"] - - df = df.sort_values(columns).reset_index(drop=True) - expected_df = expected_df.sort_values(columns).reset_index(drop=True) - assert_frame_equal(df, expected_df, **kwargs) +from tests.utils import assert_eq def test_over_with_sorting(c, user_table_1): - df = c.sql( + return_df = c.sql( """ SELECT user_id, @@ -20,36 +13,31 @@ def test_over_with_sorting(c, user_table_1): FROM user_table_1 """ ) - df = df.compute() + expected_df = user_table_1.sort_values(["user_id", "b"]) + expected_df["R"] = [1, 2, 3, 4] - expected_df = pd.DataFrame( - {"user_id": user_table_1.user_id, "b": user_table_1.b, "R": [3, 1, 2, 4]} - ) - expected_df["R"] = expected_df["R"].astype("Int64") - assert_frame_equal_after_sorting(df, expected_df, columns=["user_id", "b"]) + assert_eq(return_df, expected_df, check_dtype=False, check_index=False) def test_over_with_partitioning(c, user_table_2): - df = c.sql( + return_df = c.sql( """ SELECT user_id, c, ROW_NUMBER() OVER (PARTITION BY c) AS R FROM user_table_2 + ORDER BY user_id, c """ ) - df = df.compute() + expected_df = user_table_2.sort_values(["user_id", "c"]) + expected_df["R"] = [1, 1, 1, 1] - expected_df = pd.DataFrame( - {"user_id": user_table_2.user_id, "c": user_table_2.c, "R": [1, 1, 1, 1]} - ) - expected_df["R"] = expected_df["R"].astype("Int64") - assert_frame_equal_after_sorting(df, expected_df, columns=["user_id", "c"]) + assert_eq(return_df, expected_df, check_dtype=False, check_index=False) def test_over_with_grouping_and_sort(c, user_table_1): - df = c.sql( + return_df = c.sql( """ SELECT user_id, @@ -58,17 +46,14 @@ def test_over_with_grouping_and_sort(c, user_table_1): FROM user_table_1 """ ) - df = df.compute() + expected_df = user_table_1.sort_values(["user_id", "b"]) + expected_df["R"] = [1, 1, 2, 1] - expected_df = pd.DataFrame( - {"user_id": user_table_1.user_id, "b": user_table_1.b, "R": [2, 1, 1, 1]} - ) - expected_df["R"] = expected_df["R"].astype("Int64") - assert_frame_equal_after_sorting(df, expected_df, columns=["user_id", "b"]) + assert_eq(return_df, expected_df, check_dtype=False, check_index=False) def test_over_with_different(c, user_table_1): - df = c.sql( + return_df = c.sql( """ SELECT user_id, @@ -78,8 +63,6 @@ def test_over_with_different(c, user_table_1): FROM user_table_1 """ ) - df = df.compute() - expected_df = pd.DataFrame( { "user_id": user_table_1.user_id, @@ -88,14 +71,12 @@ def test_over_with_different(c, user_table_1): "R2": [3, 1, 2, 4], } ) - for col in ["R1", "R2"]: - expected_df[col] = expected_df[col].astype("Int64") - assert_frame_equal_after_sorting(df, expected_df, columns=["user_id", "b"]) + assert_eq(return_df, expected_df, check_dtype=False, check_index=False) def test_over_calls(c, user_table_1): - df = c.sql( + return_df = c.sql( """ SELECT user_id, @@ -113,8 +94,6 @@ def test_over_calls(c, user_table_1): FROM user_table_1 """ ) - df = df.compute() - expected_df = pd.DataFrame( { "user_id": user_table_1.user_id, @@ -131,20 +110,15 @@ def test_over_calls(c, user_table_1): "O9": [1, 3, 1, 3], } ) - for col in expected_df.columns: - if col in ["06", "user_id", "b"]: - continue - expected_df[col] = expected_df[col].astype("Int64") - expected_df["O6"] = expected_df["O6"].astype("Float64") - assert_frame_equal_after_sorting(df, expected_df, columns=["user_id", "b"]) + assert_eq(return_df, expected_df, check_dtype=False, check_index=False) def test_over_with_windows(c): - df = pd.DataFrame({"a": range(5)}) - c.create_table("tmp", df) + tmp_df = pd.DataFrame({"a": range(5)}) + c.create_table("tmp", tmp_df) - df = c.sql( + return_df = c.sql( """ SELECT a, @@ -161,11 +135,9 @@ def test_over_with_windows(c): FROM tmp """ ) - df = df.compute() - expected_df = pd.DataFrame( { - "a": df.a, + "a": return_df.a, "O1": [0, 1, 3, 6, 9], "O2": [6, 10, 10, 10, 9], "O3": [10, 10, 10, 10, 9], @@ -178,9 +150,5 @@ def test_over_with_windows(c): "O10": [None, 0, 1, 3, 6], } ) - for col in expected_df.columns: - if col in ["a"]: - continue - expected_df[col] = expected_df[col].astype("Int64") - assert_frame_equal_after_sorting(df, expected_df, columns=["a"]) + assert_eq(return_df, expected_df, check_dtype=False, check_index=False) diff --git a/tests/integration/test_rex.py b/tests/integration/test_rex.py index aa6096b9e..8e38c94ac 100644 --- a/tests/integration/test_rex.py +++ b/tests/integration/test_rex.py @@ -4,7 +4,8 @@ import numpy as np import pandas as pd import pytest -from pandas.testing import assert_frame_equal + +from tests.utils import assert_eq @pytest.mark.xfail( @@ -28,8 +29,6 @@ def test_case(c, df): FROM df """ ) - result_df = result_df.compute() - expected_df = pd.DataFrame(index=df.index) expected_df["S1"] = df.a.apply(lambda a: 1 if a == 3 else pd.NA) expected_df["S2"] = df.a.apply(lambda a: a if a > 0 else 1) @@ -40,8 +39,9 @@ def test_case(c, df): ) expected_df["S6"] = df.a.apply(lambda a: 42 if ((a < 2) or (3 < a < 4)) else 47) expected_df["S7"] = df.a.apply(lambda a: 1 if (1 < a <= 4) else 0) + # Do not check dtypes, as pandas versions are inconsistent here - assert_frame_equal(result_df, expected_df, check_dtype=False) + assert_eq(result_df, expected_df, check_dtype=False) def test_literals(c): @@ -55,7 +55,6 @@ def test_literals(c): INTERVAL '1' DAY AS "IN" """ ) - df = df.compute() expected_df = pd.DataFrame( { @@ -68,7 +67,7 @@ def test_literals(c): "IN": [pd.to_timedelta("1d")], } ) - assert_frame_equal(df, expected_df) + assert_eq(df, expected_df) def test_literal_null(c): @@ -77,49 +76,28 @@ def test_literal_null(c): SELECT NULL AS "N", 1 + NULL AS "I" """ ) - df = df.compute() expected_df = pd.DataFrame({"N": [pd.NA], "I": [pd.NA]}) expected_df["I"] = expected_df["I"].astype("Int32") - assert_frame_equal(df, expected_df) + assert_eq(df, expected_df) -def test_random(c, df): - result_df = c.sql( - """ - SELECT RAND(0) AS "0", RAND_INTEGER(1, 10) AS "1" - """ - ) - result_df = result_df.compute() +def test_random(c): + query = 'SELECT RAND(0) AS "0", RAND_INTEGER(0, 10) AS "1"' - # As the seed is fixed, this should always give the same results - expected_df = pd.DataFrame({"0": [0.26183678695392976], "1": [8]}) - expected_df["1"] = expected_df["1"].astype("Int32") - assert_frame_equal(result_df, expected_df) + result_df = c.sql(query) - result_df = c.sql( - """ - SELECT RAND(42) AS "R" FROM df WHERE RAND(0) < b - """ - ) + # assert that repeated queries give the same result + assert_eq(result_df, c.sql(query)) + + # assert output result_df = result_df.compute() - assert len(result_df) == 659 - assert list(result_df["R"].iloc[:5]) == [ - 0.5276488824980542, - 0.17861463145673728, - 0.33764733440490524, - 0.6590485298464198, - 0.08554137165307785, - ] + assert result_df["0"].dtype == "float64" + assert result_df["1"].dtype == "Int32" - # If we do not fix the seed, we can just test if it works at all - result_df = c.sql( - """ - SELECT RAND() AS "0", RAND_INTEGER(10) AS "1" - """ - ) - result_df = result_df.compute() + assert 0 <= result_df["0"][0] < 1 + assert 0 <= result_df["1"][0] < 10 @pytest.mark.parametrize( @@ -139,10 +117,9 @@ def test_not(c, input_table, request): WHERE NOT a LIKE '%normal%' """ ) - df = df.compute() expected_df = string_table[~string_table.a.str.contains("normal")] - dd.assert_eq(df, expected_df) + assert_eq(df, expected_df) def test_operators(c, df): @@ -163,7 +140,6 @@ def test_operators(c, df): FROM df """ ) - result_df = result_df.compute() expected_df = pd.DataFrame(index=df.index) expected_df["m"] = df["a"] * df["b"] @@ -177,7 +153,7 @@ def test_operators(c, df): expected_df["l"] = df["a"] < df["b"] expected_df["le"] = df["a"] <= df["b"] expected_df["n"] = df["a"] != df["b"] - assert_frame_equal(result_df, expected_df) + assert_eq(result_df, expected_df) @pytest.mark.parametrize( @@ -198,26 +174,22 @@ def test_operators(c, df): ) def test_like(c, input_table, gpu, request): string_table = request.getfixturevalue(input_table) - if gpu: - xd = pytest.importorskip("cudf") - else: - xd = pd df = c.sql( f""" SELECT * FROM {input_table} WHERE a SIMILAR TO '%n[a-z]rmal st_i%' """ - ).compute() + ) - dd.assert_eq(df, string_table.iloc[[0]]) + assert_eq(df, string_table.iloc[[0]]) df = c.sql( f""" SELECT * FROM {input_table} WHERE a LIKE '%n[a-z]rmal st_i%' """ - ).compute() + ) assert len(df) == 0 @@ -226,47 +198,47 @@ def test_like(c, input_table, gpu, request): SELECT * FROM {input_table} WHERE a LIKE 'Ä%Ä_Ä%' ESCAPE 'Ä' """ - ).compute() + ) - dd.assert_eq(df, string_table.iloc[[1]]) + assert_eq(df, string_table.iloc[[1]]) df = c.sql( f""" SELECT * FROM {input_table} WHERE a SIMILAR TO '^|()-*r[r]$' ESCAPE 'r' """ - ).compute() + ) - dd.assert_eq(df, string_table.iloc[[2]]) + assert_eq(df, string_table.iloc[[2]]) df = c.sql( f""" SELECT * FROM {input_table} WHERE a LIKE '^|()-*r[r]$' ESCAPE 'r' """ - ).compute() + ) - dd.assert_eq(df, string_table.iloc[[2]]) + assert_eq(df, string_table.iloc[[2]]) df = c.sql( f""" SELECT * FROM {input_table} WHERE a LIKE '%_' ESCAPE 'r' """ - ).compute() + ) - dd.assert_eq(df, string_table) + assert_eq(df, string_table) - string_table2 = xd.DataFrame({"b": ["a", "b", None, pd.NA, float("nan")]}) - c.register_dask_table(dd.from_pandas(string_table2, npartitions=1), "string_table2") + string_table2 = pd.DataFrame({"b": ["a", "b", None, pd.NA, float("nan")]}) + c.create_table("string_table2", string_table2, gpu=gpu) df = c.sql( """ SELECT * FROM string_table2 WHERE b LIKE 'b' """ - ).compute() + ) - dd.assert_eq(df, string_table2.iloc[[1]]) + assert_eq(df, string_table2.iloc[[1]]) def test_null(c): @@ -277,13 +249,13 @@ def test_null(c): c IS NULL AS n FROM user_table_nan """ - ).compute() + ) expected_df = pd.DataFrame(index=[0, 1, 2]) expected_df["nn"] = [True, False, True] expected_df["nn"] = expected_df["nn"].astype("boolean") expected_df["n"] = [False, True, False] - assert_frame_equal(df, expected_df) + assert_eq(df, expected_df) df = c.sql( """ @@ -292,13 +264,13 @@ def test_null(c): a IS NULL AS n FROM string_table """ - ).compute() + ) expected_df = pd.DataFrame(index=[0, 1, 2]) expected_df["nn"] = [True, True, True] expected_df["nn"] = expected_df["nn"].astype("boolean") expected_df["n"] = [False, False, False] - assert_frame_equal(df, expected_df) + assert_eq(df, expected_df) def test_boolean_operations(c): @@ -318,7 +290,7 @@ def test_boolean_operations(c): b IS UNKNOWN AS u, b IS NOT UNKNOWN AS nu FROM df""" - ).compute() + ) expected_df = pd.DataFrame( { @@ -334,7 +306,7 @@ def test_boolean_operations(c): expected_df["nt"] = expected_df["nt"].astype("boolean") expected_df["nf"] = expected_df["nf"].astype("boolean") expected_df["nu"] = expected_df["nu"].astype("boolean") - assert_frame_equal(df, expected_df) + assert_eq(df, expected_df) def test_math_operations(c, df): @@ -367,7 +339,7 @@ def test_math_operations(c, df): , TRUNCATE(b) AS "truncate" FROM df """ - ).compute() + ) expected_df = pd.DataFrame(index=df.index) expected_df["abs"] = df.b.abs() @@ -394,7 +366,7 @@ def test_math_operations(c, df): expected_df["sin"] = np.sin(df.b) expected_df["tan"] = np.tan(df.b) expected_df["truncate"] = np.trunc(df.b) - assert_frame_equal(result_df, expected_df) + assert_eq(result_df, expected_df) def test_integer_div(c, df_simple): @@ -406,7 +378,7 @@ def test_integer_div(c, df_simple): 1.0 / a AS c FROM df_simple """ - ).compute() + ) expected_df = pd.DataFrame(index=df_simple.index) expected_df["a"] = [1, 0, 0] @@ -414,7 +386,7 @@ def test_integer_div(c, df_simple): expected_df["b"] = [0, 1, 1] expected_df["b"] = expected_df["b"].astype("Int64") expected_df["c"] = [1.0, 0.5, 0.333333] - assert_frame_equal(df, expected_df) + assert_eq(df, expected_df) def test_subqueries(c, user_table_1, user_table_2): @@ -431,13 +403,10 @@ def test_subqueries(c, user_table_1, user_table_2): user_table_1.b = user_table_2.c ) """ - ).compute() - - assert_frame_equal( - df.reset_index(drop=True), - user_table_2[user_table_2.c.isin(user_table_1.b)].reset_index(drop=True), ) + assert_eq(df, user_table_2[user_table_2.c.isin(user_table_1.b)], check_index=False) + @pytest.mark.parametrize("gpu", [False, pytest.param(True, marks=pytest.mark.gpu)]) def test_string_functions(c, gpu): @@ -474,10 +443,9 @@ def test_string_functions(c, gpu): FROM {input_table} """ - ).compute() + ) if gpu: - df = df.to_pandas() df = df.astype({"c": "int64", "f": "int64", "g": "int64"}) expected_df = pd.DataFrame( @@ -507,7 +475,7 @@ def test_string_functions(c, gpu): } ) - assert_frame_equal( + assert_eq( df.head(1), expected_df, ) @@ -565,7 +533,7 @@ def test_date_functions(c): FROM df """ - ).compute() + ) expected_df = pd.DataFrame( { @@ -608,7 +576,7 @@ def test_date_functions(c): } ) - assert_frame_equal(df, expected_df, check_dtype=False) + assert_eq(df, expected_df, check_dtype=False) # test exception handling with pytest.raises(NotImplementedError): @@ -618,4 +586,4 @@ def test_date_functions(c): FLOOR(d TO YEAR) as floor_to_year FROM df """ - ).compute() + ) diff --git a/tests/integration/test_sample.py b/tests/integration/test_sample.py index 889aa420b..7e7635139 100644 --- a/tests/integration/test_sample.py +++ b/tests/integration/test_sample.py @@ -1,57 +1,46 @@ def test_sample(c, df): # Fixed sample, check absolute numbers return_df = c.sql("SELECT * FROM df TABLESAMPLE SYSTEM (20) REPEATABLE (10)") - return_df = return_df.compute() assert len(return_df) == 234 return_df = c.sql("SELECT * FROM df TABLESAMPLE SYSTEM (20) REPEATABLE (11)") - return_df = return_df.compute() assert len(return_df) == 468 # Yes, that is horrible, but at least fast... return_df = c.sql("SELECT * FROM df TABLESAMPLE SYSTEM (50) REPEATABLE (10)") - return_df = return_df.compute() assert len(return_df) == 234 return_df = c.sql("SELECT * FROM df TABLESAMPLE SYSTEM (0.001) REPEATABLE (10)") - return_df = return_df.compute() assert len(return_df) == 0 return_df = c.sql("SELECT * FROM df TABLESAMPLE SYSTEM (99.999) REPEATABLE (10)") - return_df = return_df.compute() assert len(return_df) == len(df) return_df = c.sql("SELECT * FROM df TABLESAMPLE BERNOULLI (50) REPEATABLE (10)") - return_df = return_df.compute() assert len(return_df) == 350 return_df = c.sql("SELECT * FROM df TABLESAMPLE BERNOULLI (70) REPEATABLE (10)") - return_df = return_df.compute() assert len(return_df) == 490 return_df = c.sql("SELECT * FROM df TABLESAMPLE BERNOULLI (0.001) REPEATABLE (10)") - return_df = return_df.compute() assert len(return_df) == 0 return_df = c.sql("SELECT * FROM df TABLESAMPLE BERNOULLI (99.999) REPEATABLE (10)") - return_df = return_df.compute() assert len(return_df) == len(df) # Not fixed sample, can only check boundaries return_df = c.sql("SELECT * FROM df TABLESAMPLE BERNOULLI (50)") - return_df = return_df.compute() assert len(return_df) >= 0 and len(return_df) <= len(df) return_df = c.sql("SELECT * FROM df TABLESAMPLE SYSTEM (50)") - return_df = return_df.compute() assert len(return_df) >= 0 and len(return_df) <= len(df) diff --git a/tests/integration/test_schema.py b/tests/integration/test_schema.py index 50cc90c3d..33b0ddc4d 100644 --- a/tests/integration/test_schema.py +++ b/tests/integration/test_schema.py @@ -1,30 +1,30 @@ import dask.dataframe as dd import numpy as np import pytest -from pandas.testing import assert_frame_equal from dask_sql.utils import ParsingException +from tests.utils import assert_eq def test_table_schema(c, df): - original_df = c.sql("SELECT * FROM df").compute() + original_df = c.sql("SELECT * FROM df") - assert_frame_equal(original_df, c.sql("SELECT * FROM root.df").compute()) + assert_eq(original_df, c.sql("SELECT * FROM root.df")) c.sql("CREATE SCHEMA foo") - assert_frame_equal(original_df, c.sql("SELECT * FROM df").compute()) + assert_eq(original_df, c.sql("SELECT * FROM df")) c.sql('USE SCHEMA "foo"') - assert_frame_equal(original_df, c.sql("SELECT * FROM root.df").compute()) + assert_eq(original_df, c.sql("SELECT * FROM root.df")) c.sql("CREATE TABLE bar AS TABLE root.df") - assert_frame_equal(original_df, c.sql("SELECT * FROM bar").compute()) + assert_eq(original_df, c.sql("SELECT * FROM bar")) with pytest.raises(KeyError): c.sql("CREATE TABLE other.bar AS TABLE df") c.sql('USE SCHEMA "root"') - assert_frame_equal(original_df, c.sql("SELECT * FROM foo.bar").compute()) + assert_eq(original_df, c.sql("SELECT * FROM foo.bar")) with pytest.raises(ParsingException): c.sql("SELECT * FROM bar") diff --git a/tests/integration/test_select.py b/tests/integration/test_select.py index c6ce6a5a7..6f93692f9 100644 --- a/tests/integration/test_select.py +++ b/tests/integration/test_select.py @@ -1,35 +1,31 @@ -import dask.dataframe as dd import numpy as np import pandas as pd import pytest -from pandas.testing import assert_frame_equal from dask_sql.utils import ParsingException +from tests.utils import assert_eq def test_select(c, df): result_df = c.sql("SELECT * FROM df") - result_df = result_df.compute() - assert_frame_equal(result_df, df) + assert_eq(result_df, df) def test_select_alias(c, df): result_df = c.sql("SELECT a as b, b as a FROM df") - result_df = result_df.compute() expected_df = pd.DataFrame(index=df.index) expected_df["b"] = df.a expected_df["a"] = df.b - assert_frame_equal(result_df[["a", "b"]], expected_df[["a", "b"]]) + assert_eq(result_df[["a", "b"]], expected_df[["a", "b"]]) def test_select_column(c, df): result_df = c.sql("SELECT a FROM df") - result_df = result_df.compute() - assert_frame_equal(result_df, df[["a"]]) + assert_eq(result_df, df[["a"]]) def test_select_different_types(c): @@ -42,20 +38,19 @@ def test_select_different_types(c): } ) c.create_table("df", expected_df) - df = c.sql( + result_df = c.sql( """ SELECT * FROM df """ ) - df = df.compute() - assert_frame_equal(df, expected_df) + assert_eq(result_df, expected_df) def test_select_expr(c, df): result_df = c.sql("SELECT a + 1 AS a, b AS bla, a - 1 FROM df") - result_df = result_df.compute() + result_df = result_df expected_df = pd.DataFrame( { @@ -64,7 +59,7 @@ def test_select_expr(c, df): '"df"."a" - 1': df["a"] - 1, } ) - assert_frame_equal(result_df, expected_df) + assert_eq(result_df, expected_df) def test_select_of_select(c, df): @@ -78,10 +73,9 @@ def test_select_of_select(c, df): ) AS "inner" """ ) - result_df = result_df.compute() expected_df = pd.DataFrame({"e": 2 * (df["a"] - 1), "f": 2 * df["b"] - 1}) - assert_frame_equal(result_df, expected_df) + assert_eq(result_df, expected_df) def test_select_of_select_with_casing(c, df): @@ -95,13 +89,12 @@ def test_select_of_select_with_casing(c, df): ) AS "inner" """ ) - result_df = result_df.compute() expected_df = pd.DataFrame( {"AAA": df["a"] + df["b"], "aaa": 2 * df["b"], "aAa": df["a"] - 1} ) - assert_frame_equal(result_df, expected_df) + assert_eq(result_df, expected_df) def test_wrong_input(c): @@ -118,9 +111,30 @@ def test_timezones(c, datetime_table): SELECT * FROM datetime_table """ ) - result_df = result_df.compute() - assert_frame_equal(result_df, datetime_table) + assert_eq(result_df, datetime_table) + + +@pytest.mark.parametrize( + "input_table", + [ + "long_table", + pytest.param("gpu_long_table", marks=pytest.mark.gpu), + ], +) +@pytest.mark.parametrize( + "limit,offset", + [(101, 0), (200, 0), (100, 0), (100, 99), (100, 100), (101, 101), (0, 101)], +) +def test_limit(c, input_table, limit, offset, request): + long_table = request.getfixturevalue(input_table) + + if not limit: + query = f"SELECT * FROM long_table OFFSET {offset}" + else: + query = f"SELECT * FROM long_table LIMIT {limit} OFFSET {offset}" + + assert_eq(c.sql(query), long_table.iloc[offset : offset + limit if limit else None]) @pytest.mark.parametrize( @@ -153,7 +167,7 @@ def test_date_casting(c, input_table, request): expected_df["utc_timezone"].astype(" 2, df1.a, df2.a).compute(), pd.Series([3, 2, 3]), check_names=False - ) + assert_eq(op(df1.a > 2, df1.a, df2.a), pd.Series([3, 2, 3]), check_names=False) - assert_series_equal( - op(df1.a > 2, 99, df2.a).compute(), pd.Series([3, 2, 99]), check_names=False - ) + assert_eq(op(df1.a > 2, 99, df2.a), pd.Series([3, 2, 99]), check_names=False) - assert_series_equal( - op(df1.a > 2, 99, -1).compute(), pd.Series([-1, -1, 99]), check_names=False - ) + assert_eq(op(df1.a > 2, 99, -1), pd.Series([-1, -1, 99]), check_names=False) - assert_series_equal( - op(df1.a > 2, df1.a, -1).compute(), pd.Series([-1, -1, 3]), check_names=False - ) + assert_eq(op(df1.a > 2, df1.a, -1), pd.Series([-1, -1, 3]), check_names=False) assert op(True, 1, 2) == 1 assert op(False, 1, 2) == 2 @@ -59,11 +51,9 @@ def test_case(): def test_is_true(): op = call.IsTrueOperation() - assert_series_equal( - op(df1.a > 2).compute(), pd.Series([False, False, True]), check_names=False - ) - assert_series_equal( - op(df3.a).compute(), + assert_eq(op(df1.a > 2), pd.Series([False, False, True]), check_names=False) + assert_eq( + op(df3.a), pd.Series([True, False, False], dtype="boolean"), check_names=False, ) @@ -78,11 +68,9 @@ def test_is_true(): def test_is_false(): op = call.IsFalseOperation() - assert_series_equal( - op(df1.a > 2).compute(), pd.Series([True, True, False]), check_names=False - ) - assert_series_equal( - op(df3.a).compute(), + assert_eq(op(df1.a > 2), pd.Series([True, True, False]), check_names=False) + assert_eq( + op(df3.a), pd.Series([False, False, True], dtype="boolean"), check_names=False, ) @@ -121,53 +109,51 @@ def test_nan(): assert op(None) assert op(np.NaN) assert op(pd.NA) - assert_series_equal( - op(pd.Series(["a", None, "c"])), pd.Series([False, True, False]) - ) - assert_series_equal( + assert_eq(op(pd.Series(["a", None, "c"])), pd.Series([False, True, False])) + assert_eq( op(pd.Series([3, 2, np.NaN, pd.NA])), pd.Series([False, False, True, True]) ) def test_simple_ops(): - assert_series_equal( - ops_mapping["and"](df1.a >= 2, df2.a >= 2).compute(), + assert_eq( + ops_mapping["and"](df1.a >= 2, df2.a >= 2), pd.Series([False, True, False]), check_names=False, ) - assert_series_equal( - ops_mapping["or"](df1.a >= 2, df2.a >= 2).compute(), + assert_eq( + ops_mapping["or"](df1.a >= 2, df2.a >= 2), pd.Series([True, True, True]), check_names=False, ) - assert_series_equal( - ops_mapping[">="](df1.a, df2.a).compute(), + assert_eq( + ops_mapping[">="](df1.a, df2.a), pd.Series([False, True, True]), check_names=False, ) - assert_series_equal( - ops_mapping["+"](df1.a, df2.a, df1.a).compute(), + assert_eq( + ops_mapping["+"](df1.a, df2.a, df1.a), pd.Series([5, 6, 7]), check_names=False, ) def test_math_operations(): - assert_series_equal( - ops_mapping["abs"](-df1.a).compute(), + assert_eq( + ops_mapping["abs"](-df1.a), pd.Series([1, 2, 3]), check_names=False, ) - assert_series_equal( - ops_mapping["round"](df1.a).compute(), + assert_eq( + ops_mapping["round"](df1.a), pd.Series([1, 2, 3]), check_names=False, ) - assert_series_equal( - ops_mapping["floor"](df1.a).compute(), + assert_eq( + ops_mapping["floor"](df1.a), pd.Series([1.0, 2.0, 3.0]), check_names=False, ) diff --git a/tests/unit/test_context.py b/tests/unit/test_context.py index 697c0aee5..b8cfa6504 100644 --- a/tests/unit/test_context.py +++ b/tests/unit/test_context.py @@ -6,6 +6,7 @@ from dask_sql import Context from dask_sql.datacontainer import Statistics +from tests.utils import assert_eq try: import cudf @@ -66,7 +67,7 @@ def test_explain(gpu): "DaskTableScan(table=[[root, df]]): rowcount = 100.0, cumulative cost = {100.0 rows, 101.0 cpu, 0.0 io}, id = " ) - c.create_table("df", data_frame, statistics=Statistics(row_count=1337)) + c.create_table("df", data_frame, statistics=Statistics(row_count=1337), gpu=gpu) sql_string = c.explain("SELECT * FROM df") @@ -104,18 +105,18 @@ def test_sql(gpu): c.create_table("df", data_frame, gpu=gpu) result = c.sql("SELECT * FROM df") - assert isinstance(result, dd.DataFrame if not gpu else dask_cudf.DataFrame) - dd.assert_eq(result, data_frame) + assert isinstance(result, dd.DataFrame) + assert_eq(result, data_frame) result = c.sql("SELECT * FROM df", return_futures=False) - assert isinstance(result, pd.DataFrame if not gpu else cudf.DataFrame) - dd.assert_eq(result, data_frame) + assert not isinstance(result, dd.DataFrame) + assert_eq(result, data_frame) result = c.sql( "SELECT * FROM other_df", dataframes={"other_df": data_frame}, gpu=gpu ) - assert isinstance(result, dd.DataFrame if not gpu else dask_cudf.DataFrame) - dd.assert_eq(result, data_frame) + assert isinstance(result, dd.DataFrame) + assert_eq(result, data_frame) @pytest.mark.parametrize( @@ -135,7 +136,7 @@ def test_input_types(temporary_data_file, gpu): def assert_correct_output(gpu): result = c.sql("SELECT * FROM df") assert isinstance(result, dd.DataFrame if not gpu else dask_cudf.DataFrame) - dd.assert_eq(result, df) + assert_eq(result, df) c.create_table("df", df, gpu=gpu) assert_correct_output(gpu=gpu) diff --git a/tests/utils.py b/tests/utils.py new file mode 100644 index 000000000..680365d19 --- /dev/null +++ b/tests/utils.py @@ -0,0 +1,13 @@ +import os + +from dask.dataframe.utils import assert_eq as _assert_eq + +# use independent cluster for testing if it's available +address = os.getenv("DASK_SQL_TEST_SCHEDULER", None) +scheduler = "sync" if address is None else "distributed" + + +def assert_eq(*args, **kwargs): + kwargs.setdefault("scheduler", scheduler) + + return _assert_eq(*args, **kwargs) From 95b0dd045e1261d6ba80d8e9f94094952866649d Mon Sep 17 00:00:00 2001 From: Ayush Dattagupta Date: Tue, 12 Apr 2022 11:00:31 -0700 Subject: [PATCH 17/44] Set max pin on antlr4-python-runtime (#456) * Set max pin on antlr4-python-runtime due to incompatibilities with fugue_sql * update comment on antlr max pin version --- continuous_integration/environment-3.10-jdk11-dev.yaml | 2 +- continuous_integration/environment-3.10-jdk8-dev.yaml | 2 +- continuous_integration/environment-3.8-jdk11-dev.yaml | 2 +- continuous_integration/environment-3.8-jdk8-dev.yaml | 2 +- continuous_integration/environment-3.9-jdk11-dev.yaml | 2 +- continuous_integration/environment-3.9-jdk8-dev.yaml | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/continuous_integration/environment-3.10-jdk11-dev.yaml b/continuous_integration/environment-3.10-jdk11-dev.yaml index 257b33703..14ab5c230 100644 --- a/continuous_integration/environment-3.10-jdk11-dev.yaml +++ b/continuous_integration/environment-3.10-jdk11-dev.yaml @@ -4,7 +4,7 @@ channels: - nodefaults dependencies: - adagio>=0.2.3 -- antlr4-python3-runtime>=4.9.2 +- antlr4-python3-runtime>=4.9.2, <4.10.0 # Remove max pin after qpd(fugue dependency) updates their conda recipe - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 diff --git a/continuous_integration/environment-3.10-jdk8-dev.yaml b/continuous_integration/environment-3.10-jdk8-dev.yaml index 399aeb53d..a80b4755e 100644 --- a/continuous_integration/environment-3.10-jdk8-dev.yaml +++ b/continuous_integration/environment-3.10-jdk8-dev.yaml @@ -4,7 +4,7 @@ channels: - nodefaults dependencies: - adagio>=0.2.3 -- antlr4-python3-runtime>=4.9.2 +- antlr4-python3-runtime>=4.9.2, <4.10.0 # Remove max pin after qpd(fugue dependency) updates their conda recipe - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 diff --git a/continuous_integration/environment-3.8-jdk11-dev.yaml b/continuous_integration/environment-3.8-jdk11-dev.yaml index 03968298a..9071132dc 100644 --- a/continuous_integration/environment-3.8-jdk11-dev.yaml +++ b/continuous_integration/environment-3.8-jdk11-dev.yaml @@ -4,7 +4,7 @@ channels: - nodefaults dependencies: - adagio>=0.2.3 -- antlr4-python3-runtime>=4.9.2 +- antlr4-python3-runtime>=4.9.2, <4.10.0 # Remove max pin after qpd(fugue dependency) updates their conda recipe - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 diff --git a/continuous_integration/environment-3.8-jdk8-dev.yaml b/continuous_integration/environment-3.8-jdk8-dev.yaml index 46c77c06a..dff84a617 100644 --- a/continuous_integration/environment-3.8-jdk8-dev.yaml +++ b/continuous_integration/environment-3.8-jdk8-dev.yaml @@ -4,7 +4,7 @@ channels: - nodefaults dependencies: - adagio>=0.2.3 -- antlr4-python3-runtime>=4.9.2 +- antlr4-python3-runtime>=4.9.2, <4.10.0 # Remove max pin after qpd(fugue dependency) updates their conda recipe - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 diff --git a/continuous_integration/environment-3.9-jdk11-dev.yaml b/continuous_integration/environment-3.9-jdk11-dev.yaml index b3b2d0fa4..0cf3bbea6 100644 --- a/continuous_integration/environment-3.9-jdk11-dev.yaml +++ b/continuous_integration/environment-3.9-jdk11-dev.yaml @@ -4,7 +4,7 @@ channels: - nodefaults dependencies: - adagio>=0.2.3 -- antlr4-python3-runtime>=4.9.2 +- antlr4-python3-runtime>=4.9.2, <4.10.0 # Remove max pin after qpd(fugue dependency) updates their conda recipe - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 diff --git a/continuous_integration/environment-3.9-jdk8-dev.yaml b/continuous_integration/environment-3.9-jdk8-dev.yaml index 7afce7d05..52870e1e9 100644 --- a/continuous_integration/environment-3.9-jdk8-dev.yaml +++ b/continuous_integration/environment-3.9-jdk8-dev.yaml @@ -4,7 +4,7 @@ channels: - nodefaults dependencies: - adagio>=0.2.3 -- antlr4-python3-runtime>=4.9.2 +- antlr4-python3-runtime>=4.9.2, <4.10.0 # Remove max pin after qpd(fugue dependency) updates their conda recipe - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 From 031c04c6e8163f54ad26b40c84e205dcec82cec7 Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Tue, 19 Apr 2022 15:51:07 -0400 Subject: [PATCH 18/44] Move / minimize number of cudf / dask-cudf imports (#480) * Move / minimize number of cudf / dask-cudf imports * Add tests for GPU-related errors * Fix unbound local error * Fix ddf value error --- dask_sql/input_utils/dask.py | 9 +++---- dask_sql/input_utils/location.py | 9 +++---- dask_sql/input_utils/pandaslike.py | 9 +++---- dask_sql/physical/rel/logical/aggregate.py | 7 +---- dask_sql/physical/utils/sort.py | 10 +------ tests/integration/test_create.py | 31 ++++++++++++++++++++++ 6 files changed, 42 insertions(+), 33 deletions(-) diff --git a/dask_sql/input_utils/dask.py b/dask_sql/input_utils/dask.py index 2da11e701..4dbeeef63 100644 --- a/dask_sql/input_utils/dask.py +++ b/dask_sql/input_utils/dask.py @@ -4,11 +4,6 @@ from dask_sql.input_utils.base import BaseInputPlugin -try: - import dask_cudf -except ImportError: - dask_cudf = None - class DaskInputPlugin(BaseInputPlugin): """Input Plugin for Dask DataFrames, just keeping them""" @@ -27,7 +22,9 @@ def to_dc( **kwargs ): if gpu: # pragma: no cover - if not dask_cudf: + try: + import dask_cudf + except ImportError: raise ModuleNotFoundError( "Setting `gpu=True` for table creation requires dask_cudf" ) diff --git a/dask_sql/input_utils/location.py b/dask_sql/input_utils/location.py index 8cb23a444..e1db51e6b 100644 --- a/dask_sql/input_utils/location.py +++ b/dask_sql/input_utils/location.py @@ -7,11 +7,6 @@ from dask_sql.input_utils.base import BaseInputPlugin from dask_sql.input_utils.convert import InputUtil -try: - import dask_cudf -except ImportError: - dask_cudf = None - class LocationInputPlugin(BaseInputPlugin): """Input Plugin for everything, which can be read in from a file (on disk, remote etc.)""" @@ -44,7 +39,9 @@ def to_dc( format = extension.lstrip(".") try: if gpu: # pragma: no cover - if not dask_cudf: + try: + import dask_cudf + except ImportError: raise ModuleNotFoundError( "Setting `gpu=True` for table creation requires dask-cudf" ) diff --git a/dask_sql/input_utils/pandaslike.py b/dask_sql/input_utils/pandaslike.py index 32d7ff5ea..7c3b66de0 100644 --- a/dask_sql/input_utils/pandaslike.py +++ b/dask_sql/input_utils/pandaslike.py @@ -3,11 +3,6 @@ from dask_sql.input_utils.base import BaseInputPlugin -try: - import cudf -except ImportError: - cudf = None - class PandasLikeInputPlugin(BaseInputPlugin): """Input Plugin for Pandas Like DataFrames, which get converted to dask DataFrames""" @@ -30,7 +25,9 @@ def to_dc( ): npartitions = kwargs.pop("npartitions", 1) if gpu: # pragma: no cover - if not cudf: + try: + import cudf + except ImportError: raise ModuleNotFoundError( "Setting `gpu=True` for table creation requires cudf" ) diff --git a/dask_sql/physical/rel/logical/aggregate.py b/dask_sql/physical/rel/logical/aggregate.py index cae4e42ed..0dc3bc926 100644 --- a/dask_sql/physical/rel/logical/aggregate.py +++ b/dask_sql/physical/rel/logical/aggregate.py @@ -8,11 +8,6 @@ import pandas as pd from dask import config as dask_config -try: - import dask_cudf -except ImportError: - dask_cudf = None - from dask_sql.datacontainer import ColumnContainer, DataContainer from dask_sql.physical.rel.base import BaseRelPlugin from dask_sql.physical.rex.core.call import IsNullOperation @@ -83,7 +78,7 @@ def get_supported_aggregation(self, series): if pd.api.types.is_string_dtype(series.dtype): # If dask_cudf strings dtype, return built-in aggregation - if dask_cudf is not None and isinstance(series, dask_cudf.Series): + if "cudf" in str(series._partition_type): return built_in_aggregation # With pandas StringDtype built-in aggregations work diff --git a/dask_sql/physical/utils/sort.py b/dask_sql/physical/utils/sort.py index 0e4cc9d85..c2ccce3c1 100644 --- a/dask_sql/physical/utils/sort.py +++ b/dask_sql/physical/utils/sort.py @@ -6,11 +6,6 @@ from dask_sql.utils import make_pickable_without_dask_sql -try: - import dask_cudf -except ImportError: - dask_cudf = None - def apply_sort( df: dd.DataFrame, @@ -35,10 +30,7 @@ def apply_sort( # dask / dask-cudf don't support lists of ascending / null positions if len(sort_columns) == 1 or ( - dask_cudf is not None - and isinstance(df, dask_cudf.DataFrame) - and single_ascending - and single_null_first + "cudf" in str(df._partition_type) and single_ascending and single_null_first ): try: return df.sort_values( diff --git a/tests/integration/test_create.py b/tests/integration/test_create.py index 456435b7e..ef8f82fb4 100644 --- a/tests/integration/test_create.py +++ b/tests/integration/test_create.py @@ -363,3 +363,34 @@ def test_drop(c): with pytest.raises(dask_sql.utils.ParsingException): c.sql("SELECT a FROM new_table") + + +def test_create_gpu_error(c, df, temporary_data_file): + try: + import cudf + except ImportError: + cudf = None + + if cudf is not None: + pytest.skip("GPU-related import errors only need to be checked on CPU") + + with pytest.raises(ModuleNotFoundError): + c.create_table("new_table", df, gpu=True) + + with pytest.raises(ModuleNotFoundError): + c.create_table("new_table", dd.from_pandas(df, npartitions=2), gpu=True) + + df.to_csv(temporary_data_file, index=False) + + with pytest.raises(ModuleNotFoundError): + c.sql( + f""" + CREATE TABLE + new_table + WITH ( + location = '{temporary_data_file}', + format = 'csv', + gpu = True + ) + """ + ) From 48eb98398cf2eb94d157535c6a8444d670fb65e7 Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Fri, 13 May 2022 15:24:25 -0400 Subject: [PATCH 19/44] Use `map_partitions` to compute LIMIT / OFFSET (#517) * Use map_partitions to compute limit / offset * Use partition_info to extract partition_index --- dask_sql/physical/rel/logical/limit.py | 49 ++++++++++---------------- dask_sql/physical/utils/map.py | 17 --------- 2 files changed, 18 insertions(+), 48 deletions(-) delete mode 100644 dask_sql/physical/utils/map.py diff --git a/dask_sql/physical/rel/logical/limit.py b/dask_sql/physical/rel/logical/limit.py index 58cd68fe8..fb7c3a6dd 100644 --- a/dask_sql/physical/rel/logical/limit.py +++ b/dask_sql/physical/rel/logical/limit.py @@ -5,7 +5,6 @@ from dask_sql.datacontainer import DataContainer from dask_sql.physical.rel.base import BaseRelPlugin from dask_sql.physical.rex import RexConverter -from dask_sql.physical.utils.map import map_on_partition_index if TYPE_CHECKING: import dask_sql @@ -38,25 +37,18 @@ def convert( if offset: end += offset - df = self._apply_offset(df, offset, end) + df = self._apply_limit(df, offset, end) cc = self.fix_column_to_row_type(cc, rel.getRowType()) # No column type has changed, so no need to cast again return DataContainer(df, cc) - def _apply_offset(self, df: dd.DataFrame, offset: int, end: int) -> dd.DataFrame: + def _apply_limit(self, df: dd.DataFrame, offset: int, end: int) -> dd.DataFrame: """ Limit the dataframe to the window [offset, end]. - That is unfortunately, not so simple as we do not know how many - items we have in each partition. We have therefore no other way than to - calculate (!!!) the sizes of each partition. - - After that, we can create a new dataframe from the old - dataframe by calculating for each partition if and how much - it should be used. - We do this via generating our own dask computation graph as - we need to pass the partition number to the selection - function, which is not possible with normal "map_partitions". + + Unfortunately, Dask does not currently support row selection through `iloc`, so this must be done using a custom partition function. + However, it is sometimes possible to compute this window using `head` when an `offset` is not specified. """ if not offset: # We do a (hopefully) very quick check: if the first partition @@ -65,23 +57,19 @@ def _apply_offset(self, df: dd.DataFrame, offset: int, end: int) -> dd.DataFrame if first_partition_length >= end: return df.head(end, compute=False) - # First, we need to find out which partitions we want to use. - # Therefore we count the total number of entries + # compute the size of each partition + # TODO: compute `cumsum` here when dask#9067 is resolved partition_borders = df.map_partitions(lambda x: len(x)) - # Now we let each of the partitions figure out, how much it needs to return - # using these partition borders - # For this, we generate out own dask computation graph (as it does not really - # fit well with one of the already present methods). - - # (a) we define a method to be calculated on each partition - # This method returns the part of the partition, which falls between [offset, fetch] - # Please note that the dask object "partition_borders", will be turned into - # its pandas representation at this point and we can calculate the cumsum - # (which is not possible on the dask object). Recalculating it should not cost - # us much, as we assume the number of partitions is rather small. - def select_from_to(df, partition_index, partition_borders): + def limit_partition_func(df, partition_borders, partition_info=None): + """Limit the partition to values contained within the specified window, returning an empty dataframe if there are none""" + + # TODO: remove the `cumsum` call here when dask#9067 is resolved partition_borders = partition_borders.cumsum().to_dict() + partition_index = ( + partition_info["number"] if partition_info is not None else 0 + ) + this_partition_border_left = ( partition_borders[partition_index - 1] if partition_index > 0 else 0 ) @@ -101,8 +89,7 @@ def select_from_to(df, partition_index, partition_borders): return df.iloc[from_index:to_index] - # (b) Now we just need to apply the function on every partition - # We do this via the delayed interface, which seems the easiest one. - return map_on_partition_index( - df, select_from_to, partition_borders, meta=df._meta + return df.map_partitions( + limit_partition_func, + partition_borders=partition_borders, ) diff --git a/dask_sql/physical/utils/map.py b/dask_sql/physical/utils/map.py deleted file mode 100644 index 791342ccc..000000000 --- a/dask_sql/physical/utils/map.py +++ /dev/null @@ -1,17 +0,0 @@ -from typing import Any, Callable - -import dask -import dask.dataframe as dd - - -def map_on_partition_index( - df: dd.DataFrame, f: Callable, *args: Any, **kwargs: Any -) -> dd.DataFrame: - meta = kwargs.pop("meta", None) - return dd.from_delayed( - [ - dask.delayed(f)(partition, partition_number, *args, **kwargs) - for partition_number, partition in enumerate(df.partitions) - ], - meta=meta, - ) From 7b4bc55369948ae2d01af92ea5c78da13ad39e4e Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Mon, 16 May 2022 13:13:11 -0400 Subject: [PATCH 20/44] Use `dev` images for independent cluster testing (#518) * Switch to dask dev images * Use mamba for conda installs in images * Remove sleep call for installation * Use timeout / until to wait for cluster to be initialized --- .github/docker-compose.yaml | 5 +++-- .github/workflows/test.yml | 4 ++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/.github/docker-compose.yaml b/.github/docker-compose.yaml index 56ec50b47..87431cd6f 100644 --- a/.github/docker-compose.yaml +++ b/.github/docker-compose.yaml @@ -3,15 +3,16 @@ version: '3' services: dask-scheduler: container_name: dask-scheduler - image: daskdev/dask:latest + image: daskdev/dask:dev command: dask-scheduler ports: - "8786:8786" dask-worker: container_name: dask-worker - image: daskdev/dask:latest + image: daskdev/dask:dev command: dask-worker dask-scheduler:8786 environment: + USE_MAMBA: "true" EXTRA_CONDA_PACKAGES: "pyarrow>1.0.0" # required for parquet IO volumes: - /tmp:/tmp diff --git a/.github/workflows/test.yml b/.github/workflows/test.yml index b590260f7..6201afec8 100644 --- a/.github/workflows/test.yml +++ b/.github/workflows/test.yml @@ -173,8 +173,8 @@ jobs: run: | docker-compose -f .github/docker-compose.yaml up -d - # Wait for installation - sleep 40 + # periodically ping logs until a connection has been established; assume failure after 2 minutes + timeout 2m bash -c 'until docker logs dask-worker 2>&1 | grep -q "Starting established connection"; do sleep 1; done' docker logs dask-scheduler docker logs dask-worker From b58989f9cc037cca6c5562027c7efe34f83d4664 Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Mon, 16 May 2022 19:45:57 -0400 Subject: [PATCH 21/44] Add documentation for FugueSQL integrations (#523) * Add documentation for FugueSQL integrations * Minor nitpick around autodoc obj -> class --- dask_sql/integrations/fugue.py | 36 +++++++++++++++------------- docs/source/api.rst | 5 +++- docs/source/fugue.rst | 44 ++++++++++++++++++++++++++++++++++ docs/source/index.rst | 1 + 4 files changed, 69 insertions(+), 17 deletions(-) create mode 100644 docs/source/fugue.rst diff --git a/dask_sql/integrations/fugue.py b/dask_sql/integrations/fugue.py index c9e0a076f..ce685a1ee 100644 --- a/dask_sql/integrations/fugue.py +++ b/dask_sql/integrations/fugue.py @@ -73,15 +73,15 @@ def fsql_dask( register: bool = False, fugue_conf: Any = None, ) -> Dict[str, dd.DataFrame]: - """Fugue SQL utility function that can consume Context directly. Fugue SQL is a language + """FugueSQL utility function that can consume Context directly. FugueSQL is a language extending standard SQL. It makes SQL eligible to describe end to end workflows. It also enables you to invoke python extensions in the SQL like language. For more, please read - `Fugue SQl Tutorial `_ + `FugueSQL Tutorial `_ Args: - sql: (:obj:`str`): Fugue SQL statement + sql (:obj:`str`): Fugue SQL statement ctx (:class:`dask_sql.Context`): The context to operate on, defaults to None register (:obj:`bool`): Whether to register named steps back to the context (if provided), defaults to False @@ -89,26 +89,30 @@ def fsql_dask( Example: .. code-block:: python - # schema: * - def median(df:pd.DataFrame) -> pd.DataFrame: + + # define a custom prepartition function for FugueSQL + def median(df: pd.DataFrame) -> pd.DataFrame: df["y"] = df["y"].median() return df.head(1) - # Create a context with tables df1, df2 + # create a context with some tables c = Context() ... - result = fsql_dask(''' - j = SELECT df1.*, df2.x - FROM df1 INNER JOIN df2 ON df1.key = df2.key - PERSIST # using persist because j will be used twice - TAKE 5 ROWS PREPARTITION BY x PRESORT key - PRINT - TRANSFORM j PREPARTITION BY x USING median - PRINT - ''', c, register=True) + + # run a FugueSQL query using the context as input + query = ''' + j = SELECT df1.*, df2.x + FROM df1 INNER JOIN df2 ON df1.key = df2.key + PERSIST + TAKE 5 ROWS PREPARTITION BY x PRESORT key + PRINT + TRANSFORM j PREPARTITION BY x USING median + PRINT + ''' + result = fsql_dask(query, c, register=True) + assert "j" in result assert "j" in c.tables - """ _global, _local = get_caller_global_local_vars() diff --git a/docs/source/api.rst b/docs/source/api.rst index 29c4f5632..cb5407419 100644 --- a/docs/source/api.rst +++ b/docs/source/api.rst @@ -11,4 +11,7 @@ API Documentation .. autofunction:: dask_sql.cmd_loop -.. autofunction:: dask_sql.integrations.fugue.fsql +.. autoclass:: dask_sql.integrations.fugue.DaskSQLExecutionEngine + :members: + +.. autofunction:: dask_sql.integrations.fugue.fsql_dask diff --git a/docs/source/fugue.rst b/docs/source/fugue.rst new file mode 100644 index 000000000..264d19fcd --- /dev/null +++ b/docs/source/fugue.rst @@ -0,0 +1,44 @@ +FugueSQL Integrations +===================== + +`FugueSQL `_ is a related project that aims to provide a unified SQL interface for a variety of different computing frameworks, including Dask. +While it offers a SQL engine with a larger set of supported commands, this comes at the cost of slower performance when using Dask in comparison to dask-sql. +In order to offer a "best of both worlds" solution, dask-sql includes several options to integrate with FugueSQL, using its faster implementation of SQL commands when possible and falling back on FugueSQL when necessary. + +dask-sql as a FugueSQL engine +----------------------------- + +FugueSQL users unfamiliar with dask-sql can take advantage of its functionality with minimal code changes by passing :class:`dask_sql.integrations.fugue.DaskSQLExecutionEngine` into the ``FugueSQLWorkflow`` being used to execute commands. +For more information and sample usage, see `Fugue — dask-sql as a FugueSQL engine `_. + +Using FugueSQL on an existing ``Context`` +----------------------------------------- + +dask-sql users attempting to expand their SQL querying options for an existing ``Context`` can use :func:`dask_sql.integrations.fugue.fsql_dask`, which executes the provided query using FugueSQL, using the tables within the provided context as input. +The results of this query can then optionally be registered to the context: + +.. code-block:: python + + # define a custom prepartition function for FugueSQL + def median(df: pd.DataFrame) -> pd.DataFrame: + df["y"] = df["y"].median() + return df.head(1) + + # create a context with some tables + c = Context() + ... + + # run a FugueSQL query using the context as input + query = """ + j = SELECT df1.*, df2.x + FROM df1 INNER JOIN df2 ON df1.key = df2.key + PERSIST + TAKE 5 ROWS PREPARTITION BY x PRESORT key + PRINT + TRANSFORM j PREPARTITION BY x USING median + PRINT + """ + result = fsql_dask(query, c, register=True) # results aren't registered by default + + assert "j" in result # returns a dict of resulting tables + assert "j" in c.tables # results are also registered to the context diff --git a/docs/source/index.rst b/docs/source/index.rst index 8ebb80150..8a9accc99 100644 --- a/docs/source/index.rst +++ b/docs/source/index.rst @@ -98,6 +98,7 @@ For this example, we use some data loaded from disk and query it with a SQL comm api server cmd + fugue how_does_it_work configuration From cb3d9037b9c715e66238f27c68944837813ef7bb Mon Sep 17 00:00:00 2001 From: Ayush Dattagupta Date: Tue, 17 May 2022 11:15:42 -0700 Subject: [PATCH 22/44] Timestampdiff support (#495) * added timestampdiff * initial work for timestampdiff * Added test cases for timestampdiff * Update interval month dtype mapping * Add datetimesubOperator * Uncomment timestampdiff literal tests * Update logic for handling interval_months for pandas/cudf series and scalars * Add negative diff testcases, and gpu tests * Update reinterpret and timedelta to explicitly cast to int64 instead of int * Simplify cast_column_to_type mapping logic * Add scalar handling to castOperation and reuse it for reinterpret Co-authored-by: rajagurnath --- dask_sql/mappings.py | 21 ++++--- dask_sql/physical/rex/core/call.py | 65 +++++++++++++++++++--- tests/integration/test_rex.py | 89 ++++++++++++++++++++++++++++++ 3 files changed, 159 insertions(+), 16 deletions(-) diff --git a/dask_sql/mappings.py b/dask_sql/mappings.py index 500d7c79d..e59025918 100644 --- a/dask_sql/mappings.py +++ b/dask_sql/mappings.py @@ -197,6 +197,7 @@ def sql_to_python_type(sql_type: str) -> type: return pd.StringDtype() elif sql_type.startswith("INTERVAL"): return np.dtype(" SeriesOrScalar: - if not is_frame(operand): # pragma: no cover - return operand - output_type = str(rex.getType()) python_type = sql_to_python_type(output_type.upper()) @@ -715,6 +712,43 @@ def search(self, series: dd.Series, sarg: SargPythonImplementation): return conditions[0] +class DatetimeSubOperation(Operation): + """ + Datetime subtraction is a special case of the `minus` operation in calcite + which also specifies a sql interval return type for the operation. + """ + + needs_rex = True + + def __init__(self): + super().__init__(self.datetime_sub) + + def datetime_sub(self, *operands, rex=None): + output_type = str(rex.getType()) + assert output_type.startswith("INTERVAL") + interval_unit = output_type.split()[1].lower() + + subtraction_op = ReduceOperation( + operation=operator.sub, unary_operation=lambda x: -x + ) + intermediate_res = subtraction_op(*operands) + + # Special case output_type for datetime operations + if interval_unit in {"year", "quarter", "month"}: + # if interval_unit is INTERVAL YEAR, Calcite will covert to months + if not is_frame(intermediate_res): + # Numpy doesn't allow divsion by month time unit + result = intermediate_res.astype("timedelta64[M]") + # numpy -ve timedelta's are off by one vs sql when casted to month + result = result + 1 if result < 0 else result + else: + result = intermediate_res / np.timedelta64(1, "M") + else: + result = intermediate_res.astype("timedelta64[ms]") + + return result + + class RexCallPlugin(BaseRexPlugin): """ RexCall is used for expressions, which calculate something. @@ -752,6 +786,7 @@ class RexCallPlugin(BaseRexPlugin): "/int": IntDivisionOperator(), # special operations "cast": CastOperation(), + "reinterpret": CastOperation(), "case": CaseOperation(), "like": LikeOperation(), "similar to": SimilarOperation(), @@ -812,6 +847,7 @@ class RexCallPlugin(BaseRexPlugin): lambda x: x + pd.tseries.offsets.MonthEnd(1), lambda x: convert_to_datetime(x) + pd.tseries.offsets.MonthEnd(1), ), + "datetime_subtraction": DatetimeSubOperation(), } def convert( @@ -827,6 +863,8 @@ def convert( # Now use the operator name in the mapping schema_name, operator_name = context.fqn(rex.getOperator().getNameAsId()) + if special_op := check_special_operator(rex.getOperator()): + operator_name = special_op operator_name = operator_name.lower() try: @@ -850,3 +888,16 @@ def convert( return operation(*operands, **kwargs) # TODO: We have information on the typing here - we should use it + + +def check_special_operator(operator: "org.apache.calcite.sql.fun"): + """ + Check for special operator classes that have an overloaded name with other + operator type/kinds. + + eg: sqlDatetimeSubtractionOperator has the sqltype and kind of the `-` or `minus` operation. + """ + special_op_to_name = { + "org.apache.calcite.sql.fun.SqlDatetimeSubtractionOperator": "datetime_subtraction" + } + return special_op_to_name.get(get_java_class(operator), None) diff --git a/tests/integration/test_rex.py b/tests/integration/test_rex.py index 8e38c94ac..508d2694d 100644 --- a/tests/integration/test_rex.py +++ b/tests/integration/test_rex.py @@ -587,3 +587,92 @@ def test_date_functions(c): FROM df """ ) + + +@pytest.mark.parametrize("gpu", [False, pytest.param(True, marks=pytest.mark.gpu)]) +def test_timestampdiff(c, gpu): + # single value test + ts_literal1 = "2002-03-07 09:10:05.123" + ts_literal2 = "2001-06-05 10:11:06.234" + query = ( + f"SELECT timestampdiff(NANOSECOND, CAST('{ts_literal1}' AS TIMESTAMP),CAST('{ts_literal2}' AS TIMESTAMP)) as res0," + f"timestampdiff(MICROSECOND, CAST('{ts_literal1}' AS TIMESTAMP),CAST('{ts_literal2}' AS TIMESTAMP)) as res1," + f"timestampdiff(SECOND, CAST('{ts_literal1}' AS TIMESTAMP),CAST('{ts_literal2}' AS TIMESTAMP)) as res2," + f"timestampdiff(MINUTE, CAST('{ts_literal1}' AS TIMESTAMP),CAST('{ts_literal2}' AS TIMESTAMP)) as res3," + f"timestampdiff(HOUR, CAST('{ts_literal1}' AS TIMESTAMP),CAST('{ts_literal2}' AS TIMESTAMP)) as res4," + f"timestampdiff(DAY, CAST('{ts_literal1}' AS TIMESTAMP),CAST('{ts_literal2}' AS TIMESTAMP)) as res5," + f"timestampdiff(WEEK, CAST('{ts_literal1}' AS TIMESTAMP),CAST('{ts_literal2}' AS TIMESTAMP)) as res6," + f"timestampdiff(MONTH, CAST('{ts_literal1}' AS TIMESTAMP),CAST('{ts_literal2}' AS TIMESTAMP)) as res7," + f"timestampdiff(QUARTER, CAST('{ts_literal1}' AS TIMESTAMP),CAST('{ts_literal2}' AS TIMESTAMP)) as res8," + f"timestampdiff(YEAR, CAST('{ts_literal1}' AS TIMESTAMP),CAST('{ts_literal2}' AS TIMESTAMP)) as res9" + ) + df = c.sql(query) + expected_df = pd.DataFrame( + { + "res0": [-23756339_000_000_000], + "res1": [-23756339_000_000], + "res2": [-23756339], + "res3": [-395938], + "res4": [-6598], + "res5": [-274], + "res6": [-39], + "res7": [-9], + "res8": [-3], + "res9": [0], + } + ) + assert_eq(df, expected_df) + # dataframe test + + test = pd.DataFrame( + { + "a": [ + "2002-06-05 02:01:05.200", + "2002-09-01 00:00:00", + "1970-12-03 00:00:00", + ], + "b": [ + "2002-06-07 01:00:02.100", + "2003-06-05 00:00:00", + "2038-06-05 00:00:00", + ], + } + ) + + c.create_table("test", test, gpu=gpu) + query = ( + "SELECT timestampdiff(NANOSECOND, CAST(a AS TIMESTAMP), CAST(b AS TIMESTAMP)) as nanoseconds," + "timestampdiff(MICROSECOND, CAST(a AS TIMESTAMP),CAST(b AS TIMESTAMP)) as microseconds," + "timestampdiff(SECOND, CAST(a AS TIMESTAMP),CAST(b AS TIMESTAMP)) as seconds," + "timestampdiff(MINUTE, CAST(a AS TIMESTAMP),CAST(b AS TIMESTAMP)) as minutes," + "timestampdiff(HOUR, CAST(a AS TIMESTAMP),CAST(b AS TIMESTAMP)) as hours," + "timestampdiff(DAY, CAST(a AS TIMESTAMP),CAST(b AS TIMESTAMP)) as days," + "timestampdiff(WEEK, CAST(a AS TIMESTAMP),CAST(b AS TIMESTAMP)) as weeks," + "timestampdiff(MONTH, CAST(a AS TIMESTAMP),CAST(b AS TIMESTAMP)) as months," + "timestampdiff(QUARTER, CAST(a AS TIMESTAMP),CAST(b AS TIMESTAMP)) as quarters," + "timestampdiff(YEAR, CAST(a AS TIMESTAMP),CAST(b AS TIMESTAMP)) as years" + " FROM test" + ) + + ddf = c.sql(query) + + expected_df = pd.DataFrame( + { + "nanoseconds": [ + 169136_000_000_000, + 23932_800_000_000_000, + 2_130_278_400_000_000_000, + ], + "microseconds": [169136_000_000, 23932_800_000_000, 2_130_278_400_000_000], + "seconds": [169136, 23932_800, 2_130_278_400], + "minutes": [2818, 398880, 35504640], + "hours": [46, 6648, 591744], + "days": [1, 277, 24656], + "weeks": [0, 39, 3522], + "months": [0, 9, 810], + "quarters": [0, 3, 270], + "years": [0, 0, 67], + } + ) + + assert_eq(ddf, expected_df, check_dtype=False) From 8ec3ed521449dff77ce50f5231ce78dcda679bfe Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Fri, 20 May 2022 12:17:43 -0400 Subject: [PATCH 23/44] Relax jsonschema testing dependency (#546) --- continuous_integration/environment-3.10-jdk11-dev.yaml | 2 +- continuous_integration/environment-3.10-jdk8-dev.yaml | 2 +- continuous_integration/environment-3.8-jdk11-dev.yaml | 2 +- continuous_integration/environment-3.8-jdk8-dev.yaml | 2 +- continuous_integration/environment-3.9-jdk11-dev.yaml | 3 ++- continuous_integration/environment-3.9-jdk8-dev.yaml | 2 +- 6 files changed, 7 insertions(+), 6 deletions(-) diff --git a/continuous_integration/environment-3.10-jdk11-dev.yaml b/continuous_integration/environment-3.10-jdk11-dev.yaml index 14ab5c230..cdee53be1 100644 --- a/continuous_integration/environment-3.10-jdk11-dev.yaml +++ b/continuous_integration/environment-3.10-jdk11-dev.yaml @@ -14,7 +14,7 @@ dependencies: - intake>=0.6.0 - isort=5.7.0 - jpype1>=1.0.2 -- jsonschema>=4.4.0 +- jsonschema - lightgbm>=3.2.1 - maven>=3.6.0 - mlflow>=1.19.0 diff --git a/continuous_integration/environment-3.10-jdk8-dev.yaml b/continuous_integration/environment-3.10-jdk8-dev.yaml index a80b4755e..5dd991dfc 100644 --- a/continuous_integration/environment-3.10-jdk8-dev.yaml +++ b/continuous_integration/environment-3.10-jdk8-dev.yaml @@ -14,7 +14,7 @@ dependencies: - intake>=0.6.0 - isort=5.7.0 - jpype1>=1.0.2 -- jsonschema>=4.4.0 +- jsonschema - lightgbm>=3.2.1 - maven>=3.6.0 - mlflow>=1.19.0 diff --git a/continuous_integration/environment-3.8-jdk11-dev.yaml b/continuous_integration/environment-3.8-jdk11-dev.yaml index 9071132dc..f4897110b 100644 --- a/continuous_integration/environment-3.8-jdk11-dev.yaml +++ b/continuous_integration/environment-3.8-jdk11-dev.yaml @@ -14,7 +14,7 @@ dependencies: - intake>=0.6.0 - isort=5.7.0 - jpype1>=1.0.2 -- jsonschema>=4.4.0 +- jsonschema - lightgbm>=3.2.1 - maven>=3.6.0 - mlflow>=1.19.0 diff --git a/continuous_integration/environment-3.8-jdk8-dev.yaml b/continuous_integration/environment-3.8-jdk8-dev.yaml index dff84a617..9928e4c8a 100644 --- a/continuous_integration/environment-3.8-jdk8-dev.yaml +++ b/continuous_integration/environment-3.8-jdk8-dev.yaml @@ -14,7 +14,7 @@ dependencies: - intake>=0.6.0 - isort=5.7.0 - jpype1>=1.0.2 -- jsonschema>=4.4.0 +- jsonschema - lightgbm>=3.2.1 - maven>=3.6.0 - mlflow>=1.19.0 diff --git a/continuous_integration/environment-3.9-jdk11-dev.yaml b/continuous_integration/environment-3.9-jdk11-dev.yaml index 0cf3bbea6..ecca5da20 100644 --- a/continuous_integration/environment-3.9-jdk11-dev.yaml +++ b/continuous_integration/environment-3.9-jdk11-dev.yaml @@ -14,7 +14,7 @@ dependencies: - intake>=0.6.0 - isort=5.7.0 - jpype1>=1.0.2 -- jsonschema>=4.4.0 +- jsonschema - lightgbm>=3.2.1 - maven>=3.6.0 - mlflow>=1.19.0 @@ -39,5 +39,6 @@ dependencies: - triad>=0.5.4 - tzlocal>=2.1 - uvicorn>=0.11.3 +- cfn-lint>=0.4.0 - pip: - fugue[sql]>=0.5.3 diff --git a/continuous_integration/environment-3.9-jdk8-dev.yaml b/continuous_integration/environment-3.9-jdk8-dev.yaml index 52870e1e9..b2fa6d74d 100644 --- a/continuous_integration/environment-3.9-jdk8-dev.yaml +++ b/continuous_integration/environment-3.9-jdk8-dev.yaml @@ -14,7 +14,7 @@ dependencies: - intake>=0.6.0 - isort=5.7.0 - jpype1>=1.0.2 -- jsonschema>=4.4.0 +- jsonschema - lightgbm>=3.2.1 - maven>=3.6.0 - mlflow>=1.19.0 From ff4a8a543532c4ec2131cf8f299a8fc07fef3497 Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Mon, 23 May 2022 10:50:13 -0400 Subject: [PATCH 24/44] Update upstream testing workflows (#536) * Use dask nightly conda packages for upstream testing * Add independent cluster testing to nightly upstream CI [test-upstream] * Remove unnecessary dask install [test-upstream] * Remove strict channel policy to allow nightly dask installs * Use nightly Dask packages in independent cluster test [test-upstream] * Use channels argument to install Dask conda nightlies [test-upstream] * Fix channel expression * [test-upstream] * Need to add mamba update command to get dask conda nightlies * Use conda nightlies for dask-sql import test * Add import test to upstream nightly tests * [test-upstream] * Make sure we have nightly Dask for import tests [test-upstream] --- .github/cluster-upstream.yml | 21 +++++ .github/{docker-compose.yaml => cluster.yml} | 0 .github/workflows/test-upstream.yml | 99 +++++++++++++++++++- .github/workflows/test.yml | 33 ++++--- 4 files changed, 135 insertions(+), 18 deletions(-) create mode 100644 .github/cluster-upstream.yml rename .github/{docker-compose.yaml => cluster.yml} (100%) diff --git a/.github/cluster-upstream.yml b/.github/cluster-upstream.yml new file mode 100644 index 000000000..a56eb5249 --- /dev/null +++ b/.github/cluster-upstream.yml @@ -0,0 +1,21 @@ +# Docker-compose setup used during tests +version: '3' +services: + dask-scheduler: + container_name: dask-scheduler + image: daskdev/dask:dev + command: dask-scheduler + environment: + USE_MAMBA: "true" + EXTRA_CONDA_PACKAGES: "dask/label/dev::dask" + ports: + - "8786:8786" + dask-worker: + container_name: dask-worker + image: daskdev/dask:dev + command: dask-worker dask-scheduler:8786 + environment: + USE_MAMBA: "true" + EXTRA_CONDA_PACKAGES: "dask/label/dev::dask pyarrow>1.0.0" # required for parquet IO + volumes: + - /tmp:/tmp diff --git a/.github/docker-compose.yaml b/.github/cluster.yml similarity index 100% rename from .github/docker-compose.yaml rename to .github/cluster.yml diff --git a/.github/workflows/test-upstream.yml b/.github/workflows/test-upstream.yml index a515395a0..95d657d3c 100644 --- a/.github/workflows/test-upstream.yml +++ b/.github/workflows/test-upstream.yml @@ -70,6 +70,7 @@ jobs: use-mamba: true python-version: ${{ matrix.python }} channel-priority: strict + channels: dask/label/dev,conda-forge,nodefaults activate-environment: dask-sql environment-file: ${{ env.CONDA_FILE }} - name: Download the pre-build jar @@ -89,19 +90,109 @@ jobs: echo "JAVA_HOME=${{ env.CONDA }}\envs\dask-sql\Library" >> $GITHUB_ENV - name: Install upstream dev Dask / dask-ml run: | - python -m pip install --no-deps git+https://github.com/dask/dask - python -m pip install --no-deps git+https://github.com/dask/distributed + mamba update dask python -m pip install --no-deps git+https://github.com/dask/dask-ml - name: Test with pytest run: | pytest --junitxml=junit/test-results.xml --cov-report=xml -n auto tests --dist loadfile + cluster-dev: + name: "Test upstream dev in a dask cluster" + needs: build + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v2 + - name: Cache local Maven repository + uses: actions/cache@v2 + with: + path: ~/.m2/repository + key: ${{ runner.os }}-maven-v1-jdk11-${{ hashFiles('**/pom.xml') }} + - name: Set up Python + uses: conda-incubator/setup-miniconda@v2 + with: + miniforge-variant: Mambaforge + use-mamba: true + python-version: "3.8" + channel-priority: strict + channels: dask/label/dev,conda-forge,nodefaults + activate-environment: dask-sql + environment-file: continuous_integration/environment-3.8-jdk11-dev.yaml + - name: Download the pre-build jar + uses: actions/download-artifact@v1 + with: + name: jar + path: dask_sql/jar/ + - name: Install cluster dependencies + run: | + mamba install python-blosc lz4 -c conda-forge + + which python + pip list + mamba list + - name: Install upstream dev dask-ml + run: | + mamba update dask + python -m pip install --no-deps git+https://github.com/dask/dask-ml + - name: run a dask cluster + run: | + docker-compose -f .github/cluster-upstream.yml up -d + + # periodically ping logs until a connection has been established; assume failure after 2 minutes + timeout 2m bash -c 'until docker logs dask-worker 2>&1 | grep -q "Starting established connection"; do sleep 1; done' + + docker logs dask-scheduler + docker logs dask-worker + - name: Test with pytest while running an independent dask cluster + run: | + DASK_SQL_TEST_SCHEDULER="tcp://127.0.0.1:8786" pytest --junitxml=junit/test-cluster-results.xml --cov-report=xml -n auto tests --dist loadfile + + import-dev: + name: "Test importing with bare requirements and upstream dev" + needs: build + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v2 + - name: Cache local Maven repository + uses: actions/cache@v2 + with: + path: ~/.m2/repository + key: ${{ runner.os }}-maven-v1-jdk11-${{ hashFiles('**/pom.xml') }} + - name: Set up Python + uses: conda-incubator/setup-miniconda@v2 + with: + python-version: "3.8" + mamba-version: "*" + channels: dask/label/dev,conda-forge,nodefaults + channel-priority: strict + - name: Download the pre-build jar + uses: actions/download-artifact@v1 + with: + name: jar + path: dask_sql/jar/ + - name: Install upstream dev Dask / dask-ml + if: needs.detect-ci-trigger.outputs.triggered == 'true' + run: | + mamba update dask + python -m pip install --no-deps git+https://github.com/dask/dask-ml + - name: Install dependencies and nothing else + run: | + pip install -e . + + which python + pip list + mamba list + - name: Try to import dask-sql + run: | + python -c "import dask_sql; print('ok')" + report-failures: name: Open issue for upstream dev failures - needs: test-dev + needs: [test-dev, cluster-dev] if: | always() - && needs.test-dev.result == 'failure' + && ( + needs.test-dev.result == 'failure' || needs.cluster-dev.result == 'failure' + ) runs-on: ubuntu-latest defaults: run: diff --git a/.github/workflows/test.yml b/.github/workflows/test.yml index 6201afec8..693471e15 100644 --- a/.github/workflows/test.yml +++ b/.github/workflows/test.yml @@ -92,6 +92,7 @@ jobs: use-mamba: true python-version: ${{ matrix.python }} channel-priority: strict + channels: ${{ needs.detect-ci-trigger.outputs.triggered == 'true' && 'dask/label/dev,conda-forge,nodefaults' || 'conda-forge,nodefaults' }} activate-environment: dask-sql environment-file: ${{ env.CONDA_FILE }} - name: Download the pre-build jar @@ -112,8 +113,7 @@ jobs: - name: Optionally install upstream dev Dask / dask-ml if: needs.detect-ci-trigger.outputs.triggered == 'true' run: | - python -m pip install --no-deps git+https://github.com/dask/dask - python -m pip install --no-deps git+https://github.com/dask/distributed + mamba update dask python -m pip install --no-deps git+https://github.com/dask/dask-ml - name: Test with pytest run: | @@ -149,6 +149,7 @@ jobs: use-mamba: true python-version: "3.8" channel-priority: strict + channels: ${{ needs.detect-ci-trigger.outputs.triggered == 'true' && 'dask/label/dev,conda-forge,nodefaults' || 'conda-forge,nodefaults' }} activate-environment: dask-sql environment-file: continuous_integration/environment-3.8-jdk11-dev.yaml - name: Download the pre-build jar @@ -156,22 +157,27 @@ jobs: with: name: jar path: dask_sql/jar/ - - name: Install dependencies + - name: Install cluster dependencies run: | mamba install python-blosc lz4 -c conda-forge which python pip list mamba list - - name: Optionally install upstream dev Dask / dask-ml + - name: Optionally install upstream dev dask-ml if: needs.detect-ci-trigger.outputs.triggered == 'true' run: | - python -m pip install --no-deps git+https://github.com/dask/dask - python -m pip install --no-deps git+https://github.com/dask/distributed + mamba update dask python -m pip install --no-deps git+https://github.com/dask/dask-ml - name: run a dask cluster + env: + UPSTREAM: ${{ needs.detect-ci-trigger.outputs.triggered }} run: | - docker-compose -f .github/docker-compose.yaml up -d + if [[ $UPSTREAM == "true" ]]; then + docker-compose -f .github/cluster-upstream.yml up -d + else + docker-compose -f .github/cluster.yml up -d + fi # periodically ping logs until a connection has been established; assume failure after 2 minutes timeout 2m bash -c 'until docker logs dask-worker 2>&1 | grep -q "Starting established connection"; do sleep 1; done' @@ -198,13 +204,18 @@ jobs: with: python-version: "3.8" mamba-version: "*" - channels: conda-forge,defaults + channels: ${{ needs.detect-ci-trigger.outputs.triggered == 'true' && 'dask/label/dev,conda-forge,nodefaults' || 'conda-forge,nodefaults' }} channel-priority: strict - name: Download the pre-build jar uses: actions/download-artifact@v1 with: name: jar path: dask_sql/jar/ + - name: Optionally install upstream dev Dask / dask-ml + if: needs.detect-ci-trigger.outputs.triggered == 'true' + run: | + mamba update dask + python -m pip install --no-deps git+https://github.com/dask/dask-ml - name: Install dependencies and nothing else run: | pip install -e . @@ -212,12 +223,6 @@ jobs: which python pip list mamba list - - name: Optionally install upstream dev Dask / dask-ml - if: needs.detect-ci-trigger.outputs.triggered == 'true' - run: | - python -m pip install --no-deps git+https://github.com/dask/dask - python -m pip install --no-deps git+https://github.com/dask/distributed - python -m pip install --no-deps git+https://github.com/dask/dask-ml - name: Try to import dask-sql run: | python -c "import dask_sql; print('ok')" From cb55c0760b1141675561d5d0937bcd762fecacc0 Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Tue, 24 May 2022 12:45:23 -0400 Subject: [PATCH 25/44] Fix pyarrow / cloudpickle failures in cluster testing (#553) * Explicitly install libstdcxx-ng in clusters * Make pyarrow dependency consistent across testing * Make libstdcxx-ng dep a min version * Add cloudpickle to cluster dependencies * cloudpickle must be in the scheduler environment * Bump cloudpickle version * Move cloudpickle install to workers * Fix pyarrow constraint in cluster spec --- .github/cluster-upstream.yml | 5 +---- .github/cluster.yml | 2 +- continuous_integration/environment-3.10-jdk11-dev.yaml | 2 +- continuous_integration/environment-3.10-jdk8-dev.yaml | 2 +- continuous_integration/environment-3.8-jdk11-dev.yaml | 2 +- continuous_integration/environment-3.8-jdk8-dev.yaml | 2 +- continuous_integration/environment-3.9-jdk11-dev.yaml | 2 +- continuous_integration/environment-3.9-jdk8-dev.yaml | 2 +- docker/conda.txt | 2 +- docker/main.dockerfile | 2 +- setup.py | 2 +- 11 files changed, 11 insertions(+), 14 deletions(-) diff --git a/.github/cluster-upstream.yml b/.github/cluster-upstream.yml index a56eb5249..db864bedc 100644 --- a/.github/cluster-upstream.yml +++ b/.github/cluster-upstream.yml @@ -5,9 +5,6 @@ services: container_name: dask-scheduler image: daskdev/dask:dev command: dask-scheduler - environment: - USE_MAMBA: "true" - EXTRA_CONDA_PACKAGES: "dask/label/dev::dask" ports: - "8786:8786" dask-worker: @@ -16,6 +13,6 @@ services: command: dask-worker dask-scheduler:8786 environment: USE_MAMBA: "true" - EXTRA_CONDA_PACKAGES: "dask/label/dev::dask pyarrow>1.0.0" # required for parquet IO + EXTRA_CONDA_PACKAGES: "dask/label/dev::dask cloudpickle>=2.1.0 pyarrow>=1.0.0 libstdcxx-ng>=12.1.0" volumes: - /tmp:/tmp diff --git a/.github/cluster.yml b/.github/cluster.yml index 87431cd6f..d59624c7b 100644 --- a/.github/cluster.yml +++ b/.github/cluster.yml @@ -13,6 +13,6 @@ services: command: dask-worker dask-scheduler:8786 environment: USE_MAMBA: "true" - EXTRA_CONDA_PACKAGES: "pyarrow>1.0.0" # required for parquet IO + EXTRA_CONDA_PACKAGES: "cloudpickle>=2.1.0 pyarrow>=1.0.0 libstdcxx-ng>=12.1.0" volumes: - /tmp:/tmp diff --git a/continuous_integration/environment-3.10-jdk11-dev.yaml b/continuous_integration/environment-3.10-jdk11-dev.yaml index cdee53be1..d04d81130 100644 --- a/continuous_integration/environment-3.10-jdk11-dev.yaml +++ b/continuous_integration/environment-3.10-jdk11-dev.yaml @@ -26,7 +26,7 @@ dependencies: - pre-commit>=2.11.1 - prompt_toolkit>=3.0.8 - psycopg2>=2.9.1 -- pyarrow>=0.15.1 +- pyarrow>=1.0.0 - pygments>=2.7.1 - pyhive>=0.6.4 - pytest-cov>=2.10.1 diff --git a/continuous_integration/environment-3.10-jdk8-dev.yaml b/continuous_integration/environment-3.10-jdk8-dev.yaml index 5dd991dfc..4a56a9ccf 100644 --- a/continuous_integration/environment-3.10-jdk8-dev.yaml +++ b/continuous_integration/environment-3.10-jdk8-dev.yaml @@ -26,7 +26,7 @@ dependencies: - pre-commit>=2.11.1 - prompt_toolkit>=3.0.8 - psycopg2>=2.9.1 -- pyarrow>=0.15.1 +- pyarrow>=1.0.0 - pygments>=2.7.1 - pyhive>=0.6.4 - pytest-cov>=2.10.1 diff --git a/continuous_integration/environment-3.8-jdk11-dev.yaml b/continuous_integration/environment-3.8-jdk11-dev.yaml index f4897110b..0e0ef3d7d 100644 --- a/continuous_integration/environment-3.8-jdk11-dev.yaml +++ b/continuous_integration/environment-3.8-jdk11-dev.yaml @@ -26,7 +26,7 @@ dependencies: - pre-commit>=2.11.1 - prompt_toolkit>=3.0.8 - psycopg2>=2.9.1 -- pyarrow>=0.15.1 +- pyarrow>=1.0.0 - pygments>=2.7.1 - pyhive>=0.6.4 - pytest-cov>=2.10.1 diff --git a/continuous_integration/environment-3.8-jdk8-dev.yaml b/continuous_integration/environment-3.8-jdk8-dev.yaml index 9928e4c8a..9f225bed5 100644 --- a/continuous_integration/environment-3.8-jdk8-dev.yaml +++ b/continuous_integration/environment-3.8-jdk8-dev.yaml @@ -26,7 +26,7 @@ dependencies: - pre-commit>=2.11.1 - prompt_toolkit>=3.0.8 - psycopg2>=2.9.1 -- pyarrow>=0.15.1 +- pyarrow>=1.0.0 - pygments>=2.7.1 - pyhive>=0.6.4 - pytest-cov>=2.10.1 diff --git a/continuous_integration/environment-3.9-jdk11-dev.yaml b/continuous_integration/environment-3.9-jdk11-dev.yaml index ecca5da20..1c368fbbd 100644 --- a/continuous_integration/environment-3.9-jdk11-dev.yaml +++ b/continuous_integration/environment-3.9-jdk11-dev.yaml @@ -26,7 +26,7 @@ dependencies: - pre-commit>=2.11.1 - prompt_toolkit>=3.0.8 - psycopg2>=2.9.1 -- pyarrow>=0.15.1 +- pyarrow>=1.0.0 - pygments>=2.7.1 - pyhive>=0.6.4 - pytest-cov>=2.10.1 diff --git a/continuous_integration/environment-3.9-jdk8-dev.yaml b/continuous_integration/environment-3.9-jdk8-dev.yaml index b2fa6d74d..c0dbe1cae 100644 --- a/continuous_integration/environment-3.9-jdk8-dev.yaml +++ b/continuous_integration/environment-3.9-jdk8-dev.yaml @@ -26,7 +26,7 @@ dependencies: - pre-commit>=2.11.1 - prompt_toolkit>=3.0.8 - psycopg2>=2.9.1 -- pyarrow>=0.15.1 +- pyarrow>=1.0.0 - pygments>=2.7.1 - pyhive>=0.6.4 - pytest-cov>=2.10.1 diff --git a/docker/conda.txt b/docker/conda.txt index 1796da131..e40d1069d 100644 --- a/docker/conda.txt +++ b/docker/conda.txt @@ -13,7 +13,7 @@ tzlocal>=2.1 fastapi>=0.61.1 nest-asyncio>=1.4.3 uvicorn>=0.11.3 -pyarrow>=0.15.1 +pyarrow>=1.0.0 prompt_toolkit>=3.0.8 pygments>=2.7.1 dask-ml>=2022.1.22 diff --git a/docker/main.dockerfile b/docker/main.dockerfile index 2ae85e507..848948dd5 100644 --- a/docker/main.dockerfile +++ b/docker/main.dockerfile @@ -13,7 +13,7 @@ RUN conda config --add channels conda-forge \ "tzlocal>=2.1" \ "fastapi>=0.61.1" \ "uvicorn>=0.11.3" \ - "pyarrow>=0.15.1" \ + "pyarrow>=1.0.0" \ "prompt_toolkit>=3.0.8" \ "pygments>=2.7.1" \ "dask-ml>=2022.1.22" \ diff --git a/setup.py b/setup.py index f3b59bba8..be50794e1 100755 --- a/setup.py +++ b/setup.py @@ -104,7 +104,7 @@ def build(self): "pytest-cov>=2.10.1", "mock>=4.0.3", "sphinx>=3.2.1", - "pyarrow>=0.15.1", + "pyarrow>=1.0.0", "dask-ml>=2022.1.22", "scikit-learn>=0.24.2", "intake>=0.6.0", From d8302e9a39050af9c2d7e42bd089d83f35bc21eb Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Tue, 24 May 2022 14:17:56 -0400 Subject: [PATCH 26/44] Use bash -l as default entrypoint for all jobs (#552) --- .github/workflows/test-upstream.yml | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/.github/workflows/test-upstream.yml b/.github/workflows/test-upstream.yml index 95d657d3c..e2d0577bf 100644 --- a/.github/workflows/test-upstream.yml +++ b/.github/workflows/test-upstream.yml @@ -4,6 +4,11 @@ on: - cron: "0 0 * * *" # Daily “At 00:00” UTC workflow_dispatch: # allows you to trigger the workflow run manually +# Required shell entrypoint to have properly activated conda environments +defaults: + run: + shell: bash -l {0} + jobs: build: # This build step should be similar to the deploy build, to make sure we actually test @@ -11,9 +16,6 @@ jobs: name: Build the jar on ubuntu runs-on: ubuntu-latest if: github.repository == 'dask-contrib/dask-sql' - defaults: - run: - shell: bash -l {0} steps: - uses: actions/checkout@v2 - name: Cache local Maven repository @@ -45,9 +47,6 @@ jobs: runs-on: ${{ matrix.os }} env: CONDA_FILE: continuous_integration/environment-${{ matrix.python }}-jdk${{ matrix.java }}-dev.yaml - defaults: - run: - shell: bash -l {0} strategy: fail-fast: false matrix: @@ -194,9 +193,6 @@ jobs: needs.test-dev.result == 'failure' || needs.cluster-dev.result == 'failure' ) runs-on: ubuntu-latest - defaults: - run: - shell: bash steps: - uses: actions/checkout@v2 - name: Report failures From 0d0394aae22860a3af5f419621b8d794aab6019f Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Fri, 3 Jun 2022 10:05:59 -0400 Subject: [PATCH 27/44] Constrain dask/distributed for release (#563) --- continuous_integration/environment-3.10-jdk11-dev.yaml | 2 +- continuous_integration/environment-3.10-jdk8-dev.yaml | 2 +- continuous_integration/environment-3.8-jdk11-dev.yaml | 2 +- continuous_integration/environment-3.8-jdk8-dev.yaml | 2 +- continuous_integration/environment-3.9-jdk11-dev.yaml | 2 +- continuous_integration/environment-3.9-jdk8-dev.yaml | 2 +- continuous_integration/recipe/meta.yaml | 2 +- docker/conda.txt | 2 +- docs/environment.yml | 2 +- docs/requirements-docs.txt | 2 +- setup.py | 2 +- 11 files changed, 11 insertions(+), 11 deletions(-) diff --git a/continuous_integration/environment-3.10-jdk11-dev.yaml b/continuous_integration/environment-3.10-jdk11-dev.yaml index d04d81130..a7dbcf099 100644 --- a/continuous_integration/environment-3.10-jdk11-dev.yaml +++ b/continuous_integration/environment-3.10-jdk11-dev.yaml @@ -8,7 +8,7 @@ dependencies: - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 -- dask>=2022.3.0 +- dask>=2022.3.0,<=2022.5.2 - fastapi>=0.61.1 - fs>=2.4.11 - intake>=0.6.0 diff --git a/continuous_integration/environment-3.10-jdk8-dev.yaml b/continuous_integration/environment-3.10-jdk8-dev.yaml index 4a56a9ccf..24f5c7d67 100644 --- a/continuous_integration/environment-3.10-jdk8-dev.yaml +++ b/continuous_integration/environment-3.10-jdk8-dev.yaml @@ -8,7 +8,7 @@ dependencies: - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 -- dask>=2022.3.0 +- dask>=2022.3.0,<=2022.5.2 - fastapi>=0.61.1 - fs>=2.4.11 - intake>=0.6.0 diff --git a/continuous_integration/environment-3.8-jdk11-dev.yaml b/continuous_integration/environment-3.8-jdk11-dev.yaml index 0e0ef3d7d..6e1da0383 100644 --- a/continuous_integration/environment-3.8-jdk11-dev.yaml +++ b/continuous_integration/environment-3.8-jdk11-dev.yaml @@ -8,7 +8,7 @@ dependencies: - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 -- dask>=2022.3.0 +- dask>=2022.3.0,<=2022.5.2 - fastapi>=0.61.1 - fs>=2.4.11 - intake>=0.6.0 diff --git a/continuous_integration/environment-3.8-jdk8-dev.yaml b/continuous_integration/environment-3.8-jdk8-dev.yaml index 9f225bed5..968d8e242 100644 --- a/continuous_integration/environment-3.8-jdk8-dev.yaml +++ b/continuous_integration/environment-3.8-jdk8-dev.yaml @@ -8,7 +8,7 @@ dependencies: - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 -- dask>=2022.3.0 +- dask>=2022.3.0,<=2022.5.2 - fastapi>=0.61.1 - fs>=2.4.11 - intake>=0.6.0 diff --git a/continuous_integration/environment-3.9-jdk11-dev.yaml b/continuous_integration/environment-3.9-jdk11-dev.yaml index 1c368fbbd..cf20cf18d 100644 --- a/continuous_integration/environment-3.9-jdk11-dev.yaml +++ b/continuous_integration/environment-3.9-jdk11-dev.yaml @@ -8,7 +8,7 @@ dependencies: - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 -- dask>=2022.3.0 +- dask>=2022.3.0,<=2022.5.2 - fastapi>=0.61.1 - fs>=2.4.11 - intake>=0.6.0 diff --git a/continuous_integration/environment-3.9-jdk8-dev.yaml b/continuous_integration/environment-3.9-jdk8-dev.yaml index c0dbe1cae..b93c29ef2 100644 --- a/continuous_integration/environment-3.9-jdk8-dev.yaml +++ b/continuous_integration/environment-3.9-jdk8-dev.yaml @@ -8,7 +8,7 @@ dependencies: - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 -- dask>=2022.3.0 +- dask>=2022.3.0,<=2022.5.2 - fastapi>=0.61.1 - fs>=2.4.11 - intake>=0.6.0 diff --git a/continuous_integration/recipe/meta.yaml b/continuous_integration/recipe/meta.yaml index b2a40190a..00a6cfec3 100644 --- a/continuous_integration/recipe/meta.yaml +++ b/continuous_integration/recipe/meta.yaml @@ -28,7 +28,7 @@ requirements: - python >=3.8 run: - python - - dask >=2022.3.0 + - dask >=2022.3.0,<=2022.5.2 - pandas >=1.0.0 - jpype1 >=1.0.2 - openjdk >=8 diff --git a/docker/conda.txt b/docker/conda.txt index e40d1069d..d260558d8 100644 --- a/docker/conda.txt +++ b/docker/conda.txt @@ -1,5 +1,5 @@ python>=3.8 -dask>=2022.3.0 +dask>=2022.3.0,<=2022.5.2 pandas>=1.0.0 # below 1.0, there were no nullable ext. types jpype1>=1.0.2 openjdk>=8 diff --git a/docs/environment.yml b/docs/environment.yml index ffd29270b..a720013e7 100644 --- a/docs/environment.yml +++ b/docs/environment.yml @@ -8,7 +8,7 @@ dependencies: - sphinx-tabs - dask-sphinx-theme>=2.0.3 - maven>=3.6.0 - - dask>=2022.3.0 + - dask>=2022.3.0,<=2022.5.2 - pandas>=1.0.0 - fugue>=0.5.3 - jpype1>=1.0.2 diff --git a/docs/requirements-docs.txt b/docs/requirements-docs.txt index 0cc941db8..8cda7307b 100644 --- a/docs/requirements-docs.txt +++ b/docs/requirements-docs.txt @@ -1,7 +1,7 @@ sphinx>=4.0.0 sphinx-tabs dask-sphinx-theme>=2.0.3 -dask>=2022.3.0 +dask>=2022.3.0,<=2022.5.2 pandas>=1.0.0 fugue>=0.5.3 jpype1>=1.0.2 diff --git a/setup.py b/setup.py index be50794e1..f24cacb4f 100755 --- a/setup.py +++ b/setup.py @@ -87,7 +87,7 @@ def build(self): python_requires=">=3.8", setup_requires=sphinx_requirements, install_requires=[ - "dask[dataframe,distributed]>=2022.3.0", + "dask[dataframe,distributed]>=2022.3.0,<=2022.5.2", "pandas>=1.0.0", # below 1.0, there were no nullable ext. types "jpype1>=1.0.2", "fastapi>=0.61.1", From 1e881ee6007f399db7184ba1c9890089b1a5a660 Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Fri, 3 Jun 2022 14:09:50 -0400 Subject: [PATCH 28/44] Unpin dask/distributed for development (#564) * Unpin dask/distributed post release * Remove dask/distributed version ceiling --- continuous_integration/environment-3.10-jdk11-dev.yaml | 2 +- continuous_integration/environment-3.10-jdk8-dev.yaml | 2 +- continuous_integration/environment-3.8-jdk11-dev.yaml | 2 +- continuous_integration/environment-3.8-jdk8-dev.yaml | 2 +- continuous_integration/environment-3.9-jdk11-dev.yaml | 2 +- continuous_integration/environment-3.9-jdk8-dev.yaml | 2 +- continuous_integration/recipe/meta.yaml | 2 +- docker/conda.txt | 2 +- docs/environment.yml | 2 +- docs/requirements-docs.txt | 2 +- setup.py | 2 +- 11 files changed, 11 insertions(+), 11 deletions(-) diff --git a/continuous_integration/environment-3.10-jdk11-dev.yaml b/continuous_integration/environment-3.10-jdk11-dev.yaml index a7dbcf099..d04d81130 100644 --- a/continuous_integration/environment-3.10-jdk11-dev.yaml +++ b/continuous_integration/environment-3.10-jdk11-dev.yaml @@ -8,7 +8,7 @@ dependencies: - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 -- dask>=2022.3.0,<=2022.5.2 +- dask>=2022.3.0 - fastapi>=0.61.1 - fs>=2.4.11 - intake>=0.6.0 diff --git a/continuous_integration/environment-3.10-jdk8-dev.yaml b/continuous_integration/environment-3.10-jdk8-dev.yaml index 24f5c7d67..4a56a9ccf 100644 --- a/continuous_integration/environment-3.10-jdk8-dev.yaml +++ b/continuous_integration/environment-3.10-jdk8-dev.yaml @@ -8,7 +8,7 @@ dependencies: - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 -- dask>=2022.3.0,<=2022.5.2 +- dask>=2022.3.0 - fastapi>=0.61.1 - fs>=2.4.11 - intake>=0.6.0 diff --git a/continuous_integration/environment-3.8-jdk11-dev.yaml b/continuous_integration/environment-3.8-jdk11-dev.yaml index 6e1da0383..0e0ef3d7d 100644 --- a/continuous_integration/environment-3.8-jdk11-dev.yaml +++ b/continuous_integration/environment-3.8-jdk11-dev.yaml @@ -8,7 +8,7 @@ dependencies: - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 -- dask>=2022.3.0,<=2022.5.2 +- dask>=2022.3.0 - fastapi>=0.61.1 - fs>=2.4.11 - intake>=0.6.0 diff --git a/continuous_integration/environment-3.8-jdk8-dev.yaml b/continuous_integration/environment-3.8-jdk8-dev.yaml index 968d8e242..9f225bed5 100644 --- a/continuous_integration/environment-3.8-jdk8-dev.yaml +++ b/continuous_integration/environment-3.8-jdk8-dev.yaml @@ -8,7 +8,7 @@ dependencies: - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 -- dask>=2022.3.0,<=2022.5.2 +- dask>=2022.3.0 - fastapi>=0.61.1 - fs>=2.4.11 - intake>=0.6.0 diff --git a/continuous_integration/environment-3.9-jdk11-dev.yaml b/continuous_integration/environment-3.9-jdk11-dev.yaml index cf20cf18d..1c368fbbd 100644 --- a/continuous_integration/environment-3.9-jdk11-dev.yaml +++ b/continuous_integration/environment-3.9-jdk11-dev.yaml @@ -8,7 +8,7 @@ dependencies: - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 -- dask>=2022.3.0,<=2022.5.2 +- dask>=2022.3.0 - fastapi>=0.61.1 - fs>=2.4.11 - intake>=0.6.0 diff --git a/continuous_integration/environment-3.9-jdk8-dev.yaml b/continuous_integration/environment-3.9-jdk8-dev.yaml index b93c29ef2..c0dbe1cae 100644 --- a/continuous_integration/environment-3.9-jdk8-dev.yaml +++ b/continuous_integration/environment-3.9-jdk8-dev.yaml @@ -8,7 +8,7 @@ dependencies: - black=22.3.0 - ciso8601>=2.2.0 - dask-ml>=2022.1.22 -- dask>=2022.3.0,<=2022.5.2 +- dask>=2022.3.0 - fastapi>=0.61.1 - fs>=2.4.11 - intake>=0.6.0 diff --git a/continuous_integration/recipe/meta.yaml b/continuous_integration/recipe/meta.yaml index 00a6cfec3..b2a40190a 100644 --- a/continuous_integration/recipe/meta.yaml +++ b/continuous_integration/recipe/meta.yaml @@ -28,7 +28,7 @@ requirements: - python >=3.8 run: - python - - dask >=2022.3.0,<=2022.5.2 + - dask >=2022.3.0 - pandas >=1.0.0 - jpype1 >=1.0.2 - openjdk >=8 diff --git a/docker/conda.txt b/docker/conda.txt index d260558d8..e40d1069d 100644 --- a/docker/conda.txt +++ b/docker/conda.txt @@ -1,5 +1,5 @@ python>=3.8 -dask>=2022.3.0,<=2022.5.2 +dask>=2022.3.0 pandas>=1.0.0 # below 1.0, there were no nullable ext. types jpype1>=1.0.2 openjdk>=8 diff --git a/docs/environment.yml b/docs/environment.yml index a720013e7..ffd29270b 100644 --- a/docs/environment.yml +++ b/docs/environment.yml @@ -8,7 +8,7 @@ dependencies: - sphinx-tabs - dask-sphinx-theme>=2.0.3 - maven>=3.6.0 - - dask>=2022.3.0,<=2022.5.2 + - dask>=2022.3.0 - pandas>=1.0.0 - fugue>=0.5.3 - jpype1>=1.0.2 diff --git a/docs/requirements-docs.txt b/docs/requirements-docs.txt index 8cda7307b..0cc941db8 100644 --- a/docs/requirements-docs.txt +++ b/docs/requirements-docs.txt @@ -1,7 +1,7 @@ sphinx>=4.0.0 sphinx-tabs dask-sphinx-theme>=2.0.3 -dask>=2022.3.0,<=2022.5.2 +dask>=2022.3.0 pandas>=1.0.0 fugue>=0.5.3 jpype1>=1.0.2 diff --git a/setup.py b/setup.py index f24cacb4f..be50794e1 100755 --- a/setup.py +++ b/setup.py @@ -87,7 +87,7 @@ def build(self): python_requires=">=3.8", setup_requires=sphinx_requirements, install_requires=[ - "dask[dataframe,distributed]>=2022.3.0,<=2022.5.2", + "dask[dataframe,distributed]>=2022.3.0", "pandas>=1.0.0", # below 1.0, there were no nullable ext. types "jpype1>=1.0.2", "fastapi>=0.61.1", From 243c80927fc4eb3ae734db8b213fd920eae032f3 Mon Sep 17 00:00:00 2001 From: Sarah Charlotte Johnson Date: Mon, 6 Jun 2022 09:17:07 -0700 Subject: [PATCH 29/44] update dask-sphinx-theme (#567) --- docs/requirements-docs.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/requirements-docs.txt b/docs/requirements-docs.txt index 0cc941db8..3402cca82 100644 --- a/docs/requirements-docs.txt +++ b/docs/requirements-docs.txt @@ -1,6 +1,6 @@ sphinx>=4.0.0 sphinx-tabs -dask-sphinx-theme>=2.0.3 +dask-sphinx-theme>=3.0.0 dask>=2022.3.0 pandas>=1.0.0 fugue>=0.5.3 From ec3d5da4adcda20c1a77fae07215b53408a4f43a Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Tue, 7 Jun 2022 14:49:03 -0400 Subject: [PATCH 30/44] Make sure scheduler has Dask nightlies in upstream cluster testing (#573) * Make sure scheduler has Dask nightlies in upstream cluster testing * empty commit to [test-upstream] --- .github/cluster-upstream.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.github/cluster-upstream.yml b/.github/cluster-upstream.yml index db864bedc..e10582aa1 100644 --- a/.github/cluster-upstream.yml +++ b/.github/cluster-upstream.yml @@ -5,6 +5,9 @@ services: container_name: dask-scheduler image: daskdev/dask:dev command: dask-scheduler + environment: + USE_MAMBA: "true" + EXTRA_CONDA_PACKAGES: "dask/label/dev::dask cloudpickle>=2.1.0" ports: - "8786:8786" dask-worker: From c19315a4dc7131516db2457428926c369239d57b Mon Sep 17 00:00:00 2001 From: "github-actions[bot]" <41898282+github-actions[bot]@users.noreply.github.com> Date: Tue, 7 Jun 2022 15:52:17 -0400 Subject: [PATCH 31/44] Update gpuCI `RAPIDS_VER` to `22.08` (#565) Co-authored-by: github-actions[bot] <41898282+github-actions[bot]@users.noreply.github.com> --- continuous_integration/gpuci/axis.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/continuous_integration/gpuci/axis.yaml b/continuous_integration/gpuci/axis.yaml index 41ddb56ec..a4773b2e3 100644 --- a/continuous_integration/gpuci/axis.yaml +++ b/continuous_integration/gpuci/axis.yaml @@ -8,6 +8,6 @@ LINUX_VER: - ubuntu18.04 RAPIDS_VER: -- "22.06" +- "22.08" excludes: From bc1cadc864b90bc84e73c3156feb9b053f95d809 Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Wed, 15 Jun 2022 16:22:03 -0400 Subject: [PATCH 32/44] Modify test environment pinnings to cover minimum versions (#555) * Remove black/isort deps as we prefer pre-commit * Unpin all non python/jdk dependencies * Minor package corrections for py3.9 jdk11 env * Set min version constraints for all non-testing dependencies * Pin all non-test deps for 3.8 testing * Bump sklearn min version to 1.0.0 * Bump pyarrow min version to 1.0.1 * Fix pip notation for fugue * Use unpinned deps for cluster testing for now * Add fugue deps to environments, bump pandas to 1.0.2 * Add back antlr4 version ceiling * Explicitly mark all fugue dependencies * Alter test_analyze to avoid rtol * Bump pandas to 1.0.5 to fix upstream numpy issues * Alter datetime casting util to dodge panda casting failures * Bump pandas to 1.1.0 for groupby dropna support * Simplify string dtype check for get_supported_aggregations * Add check_dtype=False back to test_group_by_nan * Bump cluster to python 3.9 * Bump fastapi to 0.69.0, resolve remaining JDBC failures * Typo - correct pandas version * Generalize test_multi_case_when's dtype check * Bump pandas to 1.1.1 to resolve flaky test failures * Constrain mlflow for windows python 3.8 testing * Selectors don't work for conda env files * Problems seem to persist in 1.1.1, bump to 1.1.2 * Remove accidental debug changes * [test-upstream] * Use python 3.9 for upstream cluster testing [test-upstream] * Updated missed pandas pinning * Unconstrain mlflow to see if Windows failures persist * Add min version for protobuf * Bump pyarrow min version to allow for newer protobuf versions --- .github/cluster-upstream.yml | 6 +- .github/cluster.yml | 6 +- .github/workflows/test-upstream.yml | 4 +- .github/workflows/test.yml | 4 +- .../environment-3.10-jdk11-dev.yaml | 52 +++++------ .../environment-3.10-jdk8-dev.yaml | 52 +++++------ .../environment-3.8-jdk11-dev.yaml | 66 ++++++------- .../environment-3.8-jdk8-dev.yaml | 66 ++++++------- .../environment-3.9-jdk11-dev.yaml | 53 ++++++----- .../environment-3.9-jdk8-dev.yaml | 52 +++++------ continuous_integration/recipe/meta.yaml | 4 +- dask_sql/physical/rel/logical/aggregate.py | 8 +- docker/conda.txt | 8 +- docker/main.dockerfile | 6 +- docs/environment.yml | 4 +- docs/requirements-docs.txt | 4 +- setup.py | 8 +- tests/integration/test_analyze.py | 19 ++-- tests/integration/test_compatibility.py | 7 +- tests/integration/test_groupby.py | 9 +- tests/integration/test_jdbc.py | 92 ++----------------- tests/integration/test_select.py | 5 +- 22 files changed, 230 insertions(+), 305 deletions(-) diff --git a/.github/cluster-upstream.yml b/.github/cluster-upstream.yml index e10582aa1..3bebbde0c 100644 --- a/.github/cluster-upstream.yml +++ b/.github/cluster-upstream.yml @@ -3,7 +3,7 @@ version: '3' services: dask-scheduler: container_name: dask-scheduler - image: daskdev/dask:dev + image: daskdev/dask:dev-py3.9 command: dask-scheduler environment: USE_MAMBA: "true" @@ -12,10 +12,10 @@ services: - "8786:8786" dask-worker: container_name: dask-worker - image: daskdev/dask:dev + image: daskdev/dask:dev-py3.9 command: dask-worker dask-scheduler:8786 environment: USE_MAMBA: "true" - EXTRA_CONDA_PACKAGES: "dask/label/dev::dask cloudpickle>=2.1.0 pyarrow>=1.0.0 libstdcxx-ng>=12.1.0" + EXTRA_CONDA_PACKAGES: "dask/label/dev::dask cloudpickle>=2.1.0 pyarrow>=3.0.0 libstdcxx-ng>=12.1.0" volumes: - /tmp:/tmp diff --git a/.github/cluster.yml b/.github/cluster.yml index d59624c7b..da9530ee7 100644 --- a/.github/cluster.yml +++ b/.github/cluster.yml @@ -3,16 +3,16 @@ version: '3' services: dask-scheduler: container_name: dask-scheduler - image: daskdev/dask:dev + image: daskdev/dask:dev-py3.9 command: dask-scheduler ports: - "8786:8786" dask-worker: container_name: dask-worker - image: daskdev/dask:dev + image: daskdev/dask:dev-py3.9 command: dask-worker dask-scheduler:8786 environment: USE_MAMBA: "true" - EXTRA_CONDA_PACKAGES: "cloudpickle>=2.1.0 pyarrow>=1.0.0 libstdcxx-ng>=12.1.0" + EXTRA_CONDA_PACKAGES: "cloudpickle>=2.1.0 pyarrow>=3.0.0 libstdcxx-ng>=12.1.0" volumes: - /tmp:/tmp diff --git a/.github/workflows/test-upstream.yml b/.github/workflows/test-upstream.yml index e2d0577bf..7f41a4d6c 100644 --- a/.github/workflows/test-upstream.yml +++ b/.github/workflows/test-upstream.yml @@ -111,11 +111,11 @@ jobs: with: miniforge-variant: Mambaforge use-mamba: true - python-version: "3.8" + python-version: "3.9" channel-priority: strict channels: dask/label/dev,conda-forge,nodefaults activate-environment: dask-sql - environment-file: continuous_integration/environment-3.8-jdk11-dev.yaml + environment-file: continuous_integration/environment-3.9-jdk11-dev.yaml - name: Download the pre-build jar uses: actions/download-artifact@v1 with: diff --git a/.github/workflows/test.yml b/.github/workflows/test.yml index 693471e15..a6f862aed 100644 --- a/.github/workflows/test.yml +++ b/.github/workflows/test.yml @@ -147,11 +147,11 @@ jobs: with: miniforge-variant: Mambaforge use-mamba: true - python-version: "3.8" + python-version: "3.9" channel-priority: strict channels: ${{ needs.detect-ci-trigger.outputs.triggered == 'true' && 'dask/label/dev,conda-forge,nodefaults' || 'conda-forge,nodefaults' }} activate-environment: dask-sql - environment-file: continuous_integration/environment-3.8-jdk11-dev.yaml + environment-file: continuous_integration/environment-3.9-jdk11-dev.yaml - name: Download the pre-build jar uses: actions/download-artifact@v1 with: diff --git a/continuous_integration/environment-3.10-jdk11-dev.yaml b/continuous_integration/environment-3.10-jdk11-dev.yaml index d04d81130..8af02bb32 100644 --- a/continuous_integration/environment-3.10-jdk11-dev.yaml +++ b/continuous_integration/environment-3.10-jdk11-dev.yaml @@ -3,41 +3,41 @@ channels: - conda-forge - nodefaults dependencies: -- adagio>=0.2.3 -- antlr4-python3-runtime>=4.9.2, <4.10.0 # Remove max pin after qpd(fugue dependency) updates their conda recipe -- black=22.3.0 -- ciso8601>=2.2.0 - dask-ml>=2022.1.22 - dask>=2022.3.0 -- fastapi>=0.61.1 -- fs>=2.4.11 +- fastapi>=0.69.0 - intake>=0.6.0 -- isort=5.7.0 - jpype1>=1.0.2 - jsonschema -- lightgbm>=3.2.1 -- maven>=3.6.0 -- mlflow>=1.19.0 -- mock>=4.0.3 -- nest-asyncio>=1.4.3 +- lightgbm +- maven +- mlflow +- mock +- nest-asyncio - openjdk=11 -- pandas>=1.0.0 # below 1.0, there were no nullable ext. types -- pip=20.2.4 -- pre-commit>=2.11.1 -- prompt_toolkit>=3.0.8 -- psycopg2>=2.9.1 -- pyarrow>=1.0.0 -- pygments>=2.7.1 -- pyhive>=0.6.4 -- pytest-cov>=2.10.1 +- pandas>=1.1.2 +- pre-commit +- prompt_toolkit +- psycopg2 +- pyarrow>=3.0.0 +- pygments +- pyhive +- pytest-cov - pytest-xdist -- pytest>=6.0.1 +- pytest - python=3.10 -- scikit-learn>=0.24.2 -- sphinx>=3.2.1 -- tpot>=0.11.7 -- triad>=0.5.4 +- scikit-learn>=1.0.0 +- sphinx +- tpot - tzlocal>=2.1 - uvicorn>=0.11.3 +# fugue dependencies; remove when we conda install fugue +- adagio +- antlr4-python3-runtime<4.10 +- ciso8601 +- fs +- pip +- qpd +- triad - pip: - fugue[sql]>=0.5.3 diff --git a/continuous_integration/environment-3.10-jdk8-dev.yaml b/continuous_integration/environment-3.10-jdk8-dev.yaml index 4a56a9ccf..ed83ed51a 100644 --- a/continuous_integration/environment-3.10-jdk8-dev.yaml +++ b/continuous_integration/environment-3.10-jdk8-dev.yaml @@ -3,41 +3,41 @@ channels: - conda-forge - nodefaults dependencies: -- adagio>=0.2.3 -- antlr4-python3-runtime>=4.9.2, <4.10.0 # Remove max pin after qpd(fugue dependency) updates their conda recipe -- black=22.3.0 -- ciso8601>=2.2.0 - dask-ml>=2022.1.22 - dask>=2022.3.0 -- fastapi>=0.61.1 -- fs>=2.4.11 +- fastapi>=0.69.0 - intake>=0.6.0 -- isort=5.7.0 - jpype1>=1.0.2 - jsonschema -- lightgbm>=3.2.1 -- maven>=3.6.0 -- mlflow>=1.19.0 -- mock>=4.0.3 -- nest-asyncio>=1.4.3 +- lightgbm +- maven +- mlflow +- mock +- nest-asyncio - openjdk=8 -- pandas>=1.0.0 # below 1.0, there were no nullable ext. types -- pip=20.2.4 -- pre-commit>=2.11.1 -- prompt_toolkit>=3.0.8 -- psycopg2>=2.9.1 -- pyarrow>=1.0.0 -- pygments>=2.7.1 -- pyhive>=0.6.4 -- pytest-cov>=2.10.1 +- pandas>=1.1.2 +- pre-commit +- prompt_toolkit +- psycopg2 +- pyarrow>=3.0.0 +- pygments +- pyhive +- pytest-cov - pytest-xdist -- pytest>=6.0.1 +- pytest - python=3.10 -- scikit-learn>=0.24.2 -- sphinx>=3.2.1 -- tpot>=0.11.7 -- triad>=0.5.4 +- scikit-learn>=1.0.0 +- sphinx +- tpot - tzlocal>=2.1 - uvicorn>=0.11.3 +# fugue dependencies; remove when we conda install fugue +- adagio +- antlr4-python3-runtime<4.10 +- ciso8601 +- fs +- pip +- qpd +- triad - pip: - fugue[sql]>=0.5.3 diff --git a/continuous_integration/environment-3.8-jdk11-dev.yaml b/continuous_integration/environment-3.8-jdk11-dev.yaml index 0e0ef3d7d..b865bb82f 100644 --- a/continuous_integration/environment-3.8-jdk11-dev.yaml +++ b/continuous_integration/environment-3.8-jdk11-dev.yaml @@ -3,41 +3,41 @@ channels: - conda-forge - nodefaults dependencies: -- adagio>=0.2.3 -- antlr4-python3-runtime>=4.9.2, <4.10.0 # Remove max pin after qpd(fugue dependency) updates their conda recipe -- black=22.3.0 -- ciso8601>=2.2.0 -- dask-ml>=2022.1.22 -- dask>=2022.3.0 -- fastapi>=0.61.1 -- fs>=2.4.11 -- intake>=0.6.0 -- isort=5.7.0 -- jpype1>=1.0.2 +- dask-ml=2022.1.22 +- dask=2022.3.0 +- fastapi=0.69.0 +- intake=0.6.0 +- jpype1=1.0.2 - jsonschema -- lightgbm>=3.2.1 -- maven>=3.6.0 -- mlflow>=1.19.0 -- mock>=4.0.3 -- nest-asyncio>=1.4.3 +- lightgbm +- maven +- mlflow +- mock +- nest-asyncio - openjdk=11 -- pandas>=1.0.0 # below 1.0, there were no nullable ext. types -- pip=20.2.4 -- pre-commit>=2.11.1 -- prompt_toolkit>=3.0.8 -- psycopg2>=2.9.1 -- pyarrow>=1.0.0 -- pygments>=2.7.1 -- pyhive>=0.6.4 -- pytest-cov>=2.10.1 +- pandas=1.1.2 +- pre-commit +- prompt_toolkit +- psycopg2 +- pyarrow=3.0.0 +- pygments +- pyhive +- pytest-cov - pytest-xdist -- pytest>=6.0.1 +- pytest - python=3.8 -- scikit-learn>=0.24.2 -- sphinx>=3.2.1 -- tpot>=0.11.7 -- triad>=0.5.4 -- tzlocal>=2.1 -- uvicorn>=0.11.3 +- scikit-learn=1.0.0 +- sphinx +- tpot +- tzlocal=2.1 +- uvicorn=0.11.3 +# fugue dependencies; remove when we conda install fugue +- adagio +- antlr4-python3-runtime<4.10 +- ciso8601 +- fs +- pip +- qpd +- triad - pip: - - fugue[sql]>=0.5.3 + - fugue[sql]==0.5.3 diff --git a/continuous_integration/environment-3.8-jdk8-dev.yaml b/continuous_integration/environment-3.8-jdk8-dev.yaml index 9f225bed5..b143fcc6c 100644 --- a/continuous_integration/environment-3.8-jdk8-dev.yaml +++ b/continuous_integration/environment-3.8-jdk8-dev.yaml @@ -3,41 +3,41 @@ channels: - conda-forge - nodefaults dependencies: -- adagio>=0.2.3 -- antlr4-python3-runtime>=4.9.2, <4.10.0 # Remove max pin after qpd(fugue dependency) updates their conda recipe -- black=22.3.0 -- ciso8601>=2.2.0 -- dask-ml>=2022.1.22 -- dask>=2022.3.0 -- fastapi>=0.61.1 -- fs>=2.4.11 -- intake>=0.6.0 -- isort=5.7.0 -- jpype1>=1.0.2 +- dask-ml=2022.1.22 +- dask=2022.3.0 +- fastapi=0.69.0 +- intake=0.6.0 +- jpype1=1.0.2 - jsonschema -- lightgbm>=3.2.1 -- maven>=3.6.0 -- mlflow>=1.19.0 -- mock>=4.0.3 -- nest-asyncio>=1.4.3 +- lightgbm +- maven +- mlflow +- mock +- nest-asyncio - openjdk=8 -- pandas>=1.0.0 # below 1.0, there were no nullable ext. types -- pip=20.2.4 -- pre-commit>=2.11.1 -- prompt_toolkit>=3.0.8 -- psycopg2>=2.9.1 -- pyarrow>=1.0.0 -- pygments>=2.7.1 -- pyhive>=0.6.4 -- pytest-cov>=2.10.1 +- pandas=1.1.2 +- pre-commit +- prompt_toolkit +- psycopg2 +- pyarrow=3.0.0 +- pygments +- pyhive +- pytest-cov - pytest-xdist -- pytest>=6.0.1 +- pytest - python=3.8 -- scikit-learn>=0.24.2 -- sphinx>=3.2.1 -- tpot>=0.11.7 -- triad>=0.5.4 -- tzlocal>=2.1 -- uvicorn>=0.11.3 +- scikit-learn=1.0.0 +- sphinx +- tpot +- tzlocal=2.1 +- uvicorn=0.11.3 +# fugue dependencies; remove when we conda install fugue +- adagio +- antlr4-python3-runtime<4.10 +- ciso8601 +- fs +- pip +- qpd +- triad - pip: - - fugue[sql]>=0.5.3 + - fugue[sql]==0.5.3 diff --git a/continuous_integration/environment-3.9-jdk11-dev.yaml b/continuous_integration/environment-3.9-jdk11-dev.yaml index 1c368fbbd..1e4430cae 100644 --- a/continuous_integration/environment-3.9-jdk11-dev.yaml +++ b/continuous_integration/environment-3.9-jdk11-dev.yaml @@ -3,42 +3,41 @@ channels: - conda-forge - nodefaults dependencies: -- adagio>=0.2.3 -- antlr4-python3-runtime>=4.9.2, <4.10.0 # Remove max pin after qpd(fugue dependency) updates their conda recipe -- black=22.3.0 -- ciso8601>=2.2.0 - dask-ml>=2022.1.22 - dask>=2022.3.0 -- fastapi>=0.61.1 -- fs>=2.4.11 +- fastapi>=0.69.0 - intake>=0.6.0 -- isort=5.7.0 - jpype1>=1.0.2 - jsonschema -- lightgbm>=3.2.1 -- maven>=3.6.0 -- mlflow>=1.19.0 -- mock>=4.0.3 -- nest-asyncio>=1.4.3 +- lightgbm +- maven +- mlflow +- mock +- nest-asyncio - openjdk=11 -- pandas>=1.0.0 # below 1.0, there were no nullable ext. types -- pip=20.2.4 -- pre-commit>=2.11.1 -- prompt_toolkit>=3.0.8 -- psycopg2>=2.9.1 -- pyarrow>=1.0.0 -- pygments>=2.7.1 -- pyhive>=0.6.4 -- pytest-cov>=2.10.1 +- pandas>=1.1.2 +- pre-commit +- prompt_toolkit +- psycopg2 +- pyarrow>=3.0.0 +- pygments +- pyhive +- pytest-cov - pytest-xdist -- pytest>=6.0.1 +- pytest - python=3.9 -- scikit-learn>=0.24.2 -- sphinx>=3.2.1 -- tpot>=0.11.7 -- triad>=0.5.4 +- scikit-learn>=1.0.0 +- sphinx +- tpot - tzlocal>=2.1 - uvicorn>=0.11.3 -- cfn-lint>=0.4.0 +# fugue dependencies; remove when we conda install fugue +- adagio +- antlr4-python3-runtime<4.10 +- ciso8601 +- fs +- pip +- qpd +- triad - pip: - fugue[sql]>=0.5.3 diff --git a/continuous_integration/environment-3.9-jdk8-dev.yaml b/continuous_integration/environment-3.9-jdk8-dev.yaml index c0dbe1cae..314aaadd0 100644 --- a/continuous_integration/environment-3.9-jdk8-dev.yaml +++ b/continuous_integration/environment-3.9-jdk8-dev.yaml @@ -3,41 +3,41 @@ channels: - conda-forge - nodefaults dependencies: -- adagio>=0.2.3 -- antlr4-python3-runtime>=4.9.2, <4.10.0 # Remove max pin after qpd(fugue dependency) updates their conda recipe -- black=22.3.0 -- ciso8601>=2.2.0 - dask-ml>=2022.1.22 - dask>=2022.3.0 -- fastapi>=0.61.1 -- fs>=2.4.11 +- fastapi>=0.69.0 - intake>=0.6.0 -- isort=5.7.0 - jpype1>=1.0.2 - jsonschema -- lightgbm>=3.2.1 -- maven>=3.6.0 -- mlflow>=1.19.0 -- mock>=4.0.3 -- nest-asyncio>=1.4.3 +- lightgbm +- maven +- mlflow +- mock +- nest-asyncio - openjdk=8 -- pandas>=1.0.0 # below 1.0, there were no nullable ext. types -- pip=20.2.4 -- pre-commit>=2.11.1 -- prompt_toolkit>=3.0.8 -- psycopg2>=2.9.1 -- pyarrow>=1.0.0 -- pygments>=2.7.1 -- pyhive>=0.6.4 -- pytest-cov>=2.10.1 +- pandas>=1.1.2 +- pre-commit +- prompt_toolkit +- psycopg2 +- pyarrow>=3.0.0 +- pygments +- pyhive +- pytest-cov - pytest-xdist -- pytest>=6.0.1 +- pytest - python=3.9 -- scikit-learn>=0.24.2 -- sphinx>=3.2.1 -- tpot>=0.11.7 -- triad>=0.5.4 +- scikit-learn>=1.0.0 +- sphinx +- tpot - tzlocal>=2.1 - uvicorn>=0.11.3 +# fugue dependencies; remove when we conda install fugue +- adagio +- antlr4-python3-runtime<4.10 +- ciso8601 +- fs +- pip +- qpd +- triad - pip: - fugue[sql]>=0.5.3 diff --git a/continuous_integration/recipe/meta.yaml b/continuous_integration/recipe/meta.yaml index b2a40190a..cd5abd580 100644 --- a/continuous_integration/recipe/meta.yaml +++ b/continuous_integration/recipe/meta.yaml @@ -29,10 +29,10 @@ requirements: run: - python - dask >=2022.3.0 - - pandas >=1.0.0 + - pandas >=1.1.2 - jpype1 >=1.0.2 - openjdk >=8 - - fastapi >=0.61.1 + - fastapi >=0.69.0 - uvicorn >=0.11.3 - tzlocal >=2.1 - prompt-toolkit diff --git a/dask_sql/physical/rel/logical/aggregate.py b/dask_sql/physical/rel/logical/aggregate.py index 0dc3bc926..5cad4476c 100644 --- a/dask_sql/physical/rel/logical/aggregate.py +++ b/dask_sql/physical/rel/logical/aggregate.py @@ -81,11 +81,8 @@ def get_supported_aggregation(self, series): if "cudf" in str(series._partition_type): return built_in_aggregation - # With pandas StringDtype built-in aggregations work - # while with pandas ObjectDtype and Nulls built-in aggregations fail - if isinstance(series, dd.Series) and isinstance( - series.dtype, pd.StringDtype - ): + # with pandas StringDtype built-in aggregations work + if isinstance(series.dtype, pd.StringDtype): return built_in_aggregation return self.custom_aggregation @@ -375,6 +372,7 @@ def _perform_aggregation( # format aggregations for Dask; also check if we can use fast path for # groupby, which is only supported if we are not using any custom aggregations + # and our pandas version support dropna for groupbys aggregations_dict = defaultdict(dict) fast_groupby = True for aggregation in aggregations: diff --git a/docker/conda.txt b/docker/conda.txt index e40d1069d..79477e40d 100644 --- a/docker/conda.txt +++ b/docker/conda.txt @@ -1,6 +1,6 @@ python>=3.8 dask>=2022.3.0 -pandas>=1.0.0 # below 1.0, there were no nullable ext. types +pandas>=1.1.2 jpype1>=1.0.2 openjdk>=8 maven>=3.6.0 @@ -10,14 +10,14 @@ pytest-xdist mock>=4.0.3 sphinx>=3.2.1 tzlocal>=2.1 -fastapi>=0.61.1 +fastapi>=0.69.0 nest-asyncio>=1.4.3 uvicorn>=0.11.3 -pyarrow>=1.0.0 +pyarrow>=3.0.0 prompt_toolkit>=3.0.8 pygments>=2.7.1 dask-ml>=2022.1.22 -scikit-learn>=0.24.2 +scikit-learn>=1.0.0 intake>=0.6.0 pre-commit>=2.11.1 black=22.3.0 diff --git a/docker/main.dockerfile b/docker/main.dockerfile index 848948dd5..cae7fb96e 100644 --- a/docker/main.dockerfile +++ b/docker/main.dockerfile @@ -11,13 +11,13 @@ RUN conda config --add channels conda-forge \ "openjdk>=11" \ "maven>=3.6.0" \ "tzlocal>=2.1" \ - "fastapi>=0.61.1" \ + "fastapi>=0.69.0" \ "uvicorn>=0.11.3" \ - "pyarrow>=1.0.0" \ + "pyarrow>=3.0.0" \ "prompt_toolkit>=3.0.8" \ "pygments>=2.7.1" \ "dask-ml>=2022.1.22" \ - "scikit-learn>=0.24.2" \ + "scikit-learn>=1.0.0" \ "intake>=0.6.0" \ && conda clean -ay diff --git a/docs/environment.yml b/docs/environment.yml index ffd29270b..f0b4eafac 100644 --- a/docs/environment.yml +++ b/docs/environment.yml @@ -9,10 +9,10 @@ dependencies: - dask-sphinx-theme>=2.0.3 - maven>=3.6.0 - dask>=2022.3.0 - - pandas>=1.0.0 + - pandas>=1.1.2 - fugue>=0.5.3 - jpype1>=1.0.2 - - fastapi>=0.61.1 + - fastapi>=0.69.0 - uvicorn>=0.11.3 - tzlocal>=2.1 - prompt_toolkit diff --git a/docs/requirements-docs.txt b/docs/requirements-docs.txt index 3402cca82..3c6c7051c 100644 --- a/docs/requirements-docs.txt +++ b/docs/requirements-docs.txt @@ -2,10 +2,10 @@ sphinx>=4.0.0 sphinx-tabs dask-sphinx-theme>=3.0.0 dask>=2022.3.0 -pandas>=1.0.0 +pandas>=1.1.2 fugue>=0.5.3 jpype1>=1.0.2 -fastapi>=0.61.1 +fastapi>=0.69.0 uvicorn>=0.11.3 tzlocal>=2.1 prompt_toolkit diff --git a/setup.py b/setup.py index be50794e1..e2bc1d540 100755 --- a/setup.py +++ b/setup.py @@ -88,9 +88,9 @@ def build(self): setup_requires=sphinx_requirements, install_requires=[ "dask[dataframe,distributed]>=2022.3.0", - "pandas>=1.0.0", # below 1.0, there were no nullable ext. types + "pandas>=1.1.2", "jpype1>=1.0.2", - "fastapi>=0.61.1", + "fastapi>=0.69.0", "uvicorn>=0.11.3", "tzlocal>=2.1", "prompt_toolkit", @@ -104,9 +104,9 @@ def build(self): "pytest-cov>=2.10.1", "mock>=4.0.3", "sphinx>=3.2.1", - "pyarrow>=1.0.0", + "pyarrow>=3.0.0", "dask-ml>=2022.1.22", - "scikit-learn>=0.24.2", + "scikit-learn>=1.0.0", "intake>=0.6.0", "pre-commit", "black==22.3.0", diff --git a/tests/integration/test_analyze.py b/tests/integration/test_analyze.py index cd51ce1b8..d3b4bd08d 100644 --- a/tests/integration/test_analyze.py +++ b/tests/integration/test_analyze.py @@ -12,11 +12,12 @@ def test_analyze(c, df): 700.0, df.a.mean(), df.a.std(), - 1.0, + df.a.min(), + # Dask's approx quantiles do not match up with pandas and must be specified explicitly + 2.0, 2.0, - 2.0, # incorrect, but what Dask gives for approx quantile - 3.0, 3.0, + df.a.max(), "double", "a", ], @@ -25,9 +26,10 @@ def test_analyze(c, df): df.b.mean(), df.b.std(), df.b.min(), - df.b.quantile(0.25), - df.b.quantile(0.5), - df.b.quantile(0.75), + # Dask's approx quantiles do not match up with pandas and must be specified explicitly + 2.73108, + 5.20286, + 7.60595, df.b.max(), "double", "b", @@ -47,9 +49,8 @@ def test_analyze(c, df): ], ) - # The percentiles are calculated only approximately, therefore we do not use exact matching - assert_eq(result_df, expected_df, rtol=0.135) + assert_eq(result_df, expected_df) result_df = c.sql("ANALYZE TABLE df COMPUTE STATISTICS FOR COLUMNS a") - assert_eq(result_df, expected_df[["a"]], rtol=0.135) + assert_eq(result_df, expected_df[["a"]]) diff --git a/tests/integration/test_compatibility.py b/tests/integration/test_compatibility.py index 63c1668b2..ec2949229 100644 --- a/tests/integration/test_compatibility.py +++ b/tests/integration/test_compatibility.py @@ -22,8 +22,11 @@ def cast_datetime_to_string(df): cols = df.select_dtypes(include=["datetime64[ns]"]).columns.tolist() - # Casting to object first as - # directly converting to string looses second precision + + if not cols: + return df + + # Casting directly to string loses second precision df[cols] = df[cols].astype("object").astype("string") return df diff --git a/tests/integration/test_groupby.py b/tests/integration/test_groupby.py index 658ad4fa2..309d25b60 100644 --- a/tests/integration/test_groupby.py +++ b/tests/integration/test_groupby.py @@ -152,13 +152,8 @@ def test_group_by_nan(c): ) expected_df = pd.DataFrame({"c": [3, float("nan"), 1]}) - # The dtype in pandas 1.0.5 and pandas 1.1.0 are different, so - # we cannot check here - assert_eq( - return_df.sort_values("c").reset_index(drop=True), - expected_df.sort_values("c").reset_index(drop=True), - check_dtype=False, - ) + # we return nullable int dtype instead of float + assert_eq(return_df, expected_df, check_dtype=False) return_df = c.sql( """ diff --git a/tests/integration/test_jdbc.py b/tests/integration/test_jdbc.py index ce216da84..62ce7d772 100644 --- a/tests/integration/test_jdbc.py +++ b/tests/integration/test_jdbc.py @@ -100,89 +100,17 @@ def test_jdbc_has_columns(app_client, c): data=f"SELECT * from system.jdbc.columns where TABLE_NAME = '{table}'", ) assert response.status_code == 200 - result = get_result_or_error(app_client, response) + client_result = get_result_or_error(app_client, response) - assert_result(result, 24, 3) - assert result["data"] == [ - [ - "", - "a_schema", - "a_table", - "A_STR", - "VARCHAR", - "VARCHAR", - "", - "", - "", - "", - "", - "", - "", - "VARCHAR", - "", - "", - "1", - "", - "", - "", - "", - "", - "", - "", - ], - [ - "", - "a_schema", - "a_table", - "AN_INT", - "INTEGER", - "INTEGER", - "", - "", - "", - "", - "", - "", - "", - "INTEGER", - "", - "", - "2", - "", - "", - "", - "", - "", - "", - "", - ], - [ - "", - "a_schema", - "a_table", - "A_FLOAT", - "FLOAT", - "FLOAT", - "", - "", - "", - "", - "", - "", - "", - "FLOAT", - "", - "", - "3", - "", - "", - "", - "", - "", - "", - "", - ], - ] + # ordering of rows isn't consistent between fastapi versions + context_result = ( + c.sql("SELECT * FROM system_jdbc.columns WHERE TABLE_NAME = 'a_table'") + .compute() + .values.tolist() + ) + + assert_result(client_result, 24, 3) + assert client_result["data"] == context_result def assert_result(result, col_len, data_len): diff --git a/tests/integration/test_select.py b/tests/integration/test_select.py index 6f93692f9..f5c4b7911 100644 --- a/tests/integration/test_select.py +++ b/tests/integration/test_select.py @@ -204,6 +204,7 @@ def test_multi_case_when(c): FROM df """ ) - expected_df = pd.DataFrame({"C": [0, 1, 1, 1, 0]}, dtype=np.int32) + expected_df = pd.DataFrame({"C": [0, 1, 1, 1, 0]}) - assert_eq(actual_df, expected_df) + # dtype varies between int32/int64 depending on pandas version + assert_eq(actual_df, expected_df, check_dtype=False) From 0db4506e3b1764937cf6775b8b6fa9e5acfbc64f Mon Sep 17 00:00:00 2001 From: ksonj Date: Wed, 15 Jun 2022 23:18:50 +0200 Subject: [PATCH 33/44] Don't move jar to local mvn repo (#579) --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index e2bc1d540..303a351fb 100755 --- a/setup.py +++ b/setup.py @@ -19,7 +19,7 @@ def install_java_libraries(dir): raise OSError( "Can not find the mvn (maven) binary. Make sure to install maven before building the jar." ) - command = [maven_command, "clean", "install", "-f", "pom.xml"] + command = [maven_command, "clean", "package", "-f", "pom.xml"] subprocess.check_call(command, cwd=os.path.join(dir, "planner")) # copy generated jar to python package From ddc26ee09aab874b0c453db71c1b5e3f2cfb4bef Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Fri, 22 Jul 2022 10:27:05 -0400 Subject: [PATCH 34/44] Add max version constraint for `fugue` (#639) * Remove antlr4-python3-runtime constraint from 3.9+ test envs * Revert "Remove antlr4-python3-runtime constraint from 3.9+ test envs" This reverts commit ef3065658b669bfbef41fc71cf6b7369b443b362. * Add max version constraint for fugue in 3.9+ envs * Constrain Fugue in remaining env/setup files * Clarify fugue constraint comments * Add pinning back to python 3.8 jdk11 tests * More reversions to python 3.8 jdk11 testing env --- continuous_integration/environment-3.10-jdk11-dev.yaml | 3 ++- continuous_integration/environment-3.10-jdk8-dev.yaml | 3 ++- continuous_integration/environment-3.9-jdk11-dev.yaml | 3 ++- continuous_integration/environment-3.9-jdk8-dev.yaml | 3 ++- docs/environment.yml | 3 ++- docs/requirements-docs.txt | 3 ++- setup.py | 3 ++- 7 files changed, 14 insertions(+), 7 deletions(-) diff --git a/continuous_integration/environment-3.10-jdk11-dev.yaml b/continuous_integration/environment-3.10-jdk11-dev.yaml index 8af02bb32..d6c45735e 100644 --- a/continuous_integration/environment-3.10-jdk11-dev.yaml +++ b/continuous_integration/environment-3.10-jdk11-dev.yaml @@ -40,4 +40,5 @@ dependencies: - qpd - triad - pip: - - fugue[sql]>=0.5.3 + # FIXME: tests are failing with fugue 0.7.0 + - fugue[sql]>=0.5.3,<0.7.0 diff --git a/continuous_integration/environment-3.10-jdk8-dev.yaml b/continuous_integration/environment-3.10-jdk8-dev.yaml index ed83ed51a..f28206297 100644 --- a/continuous_integration/environment-3.10-jdk8-dev.yaml +++ b/continuous_integration/environment-3.10-jdk8-dev.yaml @@ -40,4 +40,5 @@ dependencies: - qpd - triad - pip: - - fugue[sql]>=0.5.3 + # FIXME: tests are failing with fugue 0.7.0 + - fugue[sql]>=0.5.3,<0.7.0 diff --git a/continuous_integration/environment-3.9-jdk11-dev.yaml b/continuous_integration/environment-3.9-jdk11-dev.yaml index 1e4430cae..5a1c19777 100644 --- a/continuous_integration/environment-3.9-jdk11-dev.yaml +++ b/continuous_integration/environment-3.9-jdk11-dev.yaml @@ -40,4 +40,5 @@ dependencies: - qpd - triad - pip: - - fugue[sql]>=0.5.3 + # FIXME: tests are failing with fugue 0.7.0 + - fugue[sql]>=0.5.3,<0.7.0 diff --git a/continuous_integration/environment-3.9-jdk8-dev.yaml b/continuous_integration/environment-3.9-jdk8-dev.yaml index 314aaadd0..6f8367e51 100644 --- a/continuous_integration/environment-3.9-jdk8-dev.yaml +++ b/continuous_integration/environment-3.9-jdk8-dev.yaml @@ -40,4 +40,5 @@ dependencies: - qpd - triad - pip: - - fugue[sql]>=0.5.3 + # FIXME: tests are failing with fugue 0.7.0 + - fugue[sql]>=0.5.3,<0.7.0 diff --git a/docs/environment.yml b/docs/environment.yml index f0b4eafac..f8e0df2f8 100644 --- a/docs/environment.yml +++ b/docs/environment.yml @@ -10,7 +10,8 @@ dependencies: - maven>=3.6.0 - dask>=2022.3.0 - pandas>=1.1.2 - - fugue>=0.5.3 + # FIXME: tests are failing with fugue 0.7.0 + - fugue>=0.5.3,<0.7.0 - jpype1>=1.0.2 - fastapi>=0.69.0 - uvicorn>=0.11.3 diff --git a/docs/requirements-docs.txt b/docs/requirements-docs.txt index 3c6c7051c..09870d22e 100644 --- a/docs/requirements-docs.txt +++ b/docs/requirements-docs.txt @@ -3,7 +3,8 @@ sphinx-tabs dask-sphinx-theme>=3.0.0 dask>=2022.3.0 pandas>=1.1.2 -fugue>=0.5.3 +# FIXME: tests are failing with fugue 0.7.0 +fugue>=0.5.3,<0.7.0 jpype1>=1.0.2 fastapi>=0.69.0 uvicorn>=0.11.3 diff --git a/setup.py b/setup.py index 303a351fb..79f2247b3 100755 --- a/setup.py +++ b/setup.py @@ -112,7 +112,8 @@ def build(self): "black==22.3.0", "isort==5.7.0", ], - "fugue": ["fugue[sql]>=0.5.3"], + # FIXME: tests are failing with fugue 0.7.0 + "fugue": ["fugue[sql]>=0.5.3,<0.7.0"], }, entry_points={ "console_scripts": [ From 8a7330948a7a8603c4e25297c2038f1f7541aad2 Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Mon, 25 Jul 2022 09:42:35 -0400 Subject: [PATCH 35/44] Add environment file & documentation for GPU tests (#633) * Add gpuCI environment file * Add documentation for GPU tests / environment * Add GPU testing to docs page --- README.md | 13 +++++++++++++ continuous_integration/gpuci/environment.yaml | 17 +++++++++++++++++ docs/source/installation.rst | 13 +++++++++++++ 3 files changed, 43 insertions(+) create mode 100644 continuous_integration/gpuci/environment.yaml diff --git a/README.md b/README.md index 497a45358..7651c8196 100644 --- a/README.md +++ b/README.md @@ -135,6 +135,19 @@ You can run the tests (after installation) with pytest tests +GPU-specific tests require additional dependencies specified in `continuous_integration/gpuci/environment.yaml`. +These can be added to the development environment by running + +``` +conda env update -n dask-sql -f continuous_integration/gpuci/environment.yaml +``` + +And GPU-specific tests can be run with + +``` +pytest tests -m gpu --rungpu +``` + ## SQL Server `dask-sql` comes with a small test implementation for a SQL server. diff --git a/continuous_integration/gpuci/environment.yaml b/continuous_integration/gpuci/environment.yaml new file mode 100644 index 000000000..efceccb2f --- /dev/null +++ b/continuous_integration/gpuci/environment.yaml @@ -0,0 +1,17 @@ +name: gpuci +channels: + - rapidsai + - rapidsai-nightly + - nvidia +dependencies: + - rust>=1.60.0 + - setuptools-rust>=1.2.0 + - cudatoolkit=11.5 + - cudf=22.08 + - cuml=22.08 + - dask-cudf=22.08 + - dask-cuda=22.08 + - numpy>=1.20.0 + - ucx-proc=*=gpu + - ucx-py=0.27 + - xgboost=*=cuda_* diff --git a/docs/source/installation.rst b/docs/source/installation.rst index e082dc893..8c194fd2c 100644 --- a/docs/source/installation.rst +++ b/docs/source/installation.rst @@ -109,6 +109,19 @@ You can run the tests (after installation) with pytest tests +GPU-specific tests require additional dependencies specified in `continuous_integration/gpuci/environment.yaml`. +These can be added to the development environment by running + +.. code-block:: bash + + conda env update -n dask-sql -f continuous_integration/gpuci/environment.yaml + +And GPU-specific tests can be run with + +.. code-block:: bash + + pytest tests -m gpu --rungpu + This repository uses pre-commit hooks. To install them, call .. code-block:: bash From 20daf8981e56c9a258049926f725036809a3bfa1 Mon Sep 17 00:00:00 2001 From: brandon-b-miller <53796099+brandon-b-miller@users.noreply.github.com> Date: Tue, 26 Jul 2022 12:56:51 -0500 Subject: [PATCH 36/44] Validate UDF metadata (#641) * initial * improvements * bugfixes * Move UDF validation to registration, cache relevant info Co-authored-by: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> --- dask_sql/context.py | 22 +++++++---- dask_sql/datacontainer.py | 6 --- dask_sql/mappings.py | 5 +++ tests/integration/test_function.py | 21 +++++++--- tests/unit/test_context.py | 62 ++++++++++++++++++------------ 5 files changed, 73 insertions(+), 43 deletions(-) diff --git a/dask_sql/context.py b/dask_sql/context.py index 98cc46e21..6dc1850f1 100644 --- a/dask_sql/context.py +++ b/dask_sql/context.py @@ -748,7 +748,7 @@ def _prepare_schemas(self): logger.debug("No custom functions defined.") for function_description in schema.function_lists: name = function_description.name - sql_return_type = python_to_sql_type(function_description.return_type) + sql_return_type = function_description.return_type if function_description.aggregation: logger.debug(f"Adding function '{name}' to schema as aggregation.") dask_function = DaskAggregateFunction(name, sql_return_type) @@ -771,10 +771,7 @@ def _prepare_schemas(self): @staticmethod def _add_parameters_from_description(function_description, dask_function): for parameter in function_description.parameters: - param_name, param_type = parameter - sql_param_type = python_to_sql_type(param_type) - - dask_function.addParameter(param_name, sql_param_type, False) + dask_function.addParameter(*parameter, False) return dask_function @@ -898,9 +895,16 @@ def _register_callable( 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() @@ -920,9 +924,13 @@ def _register_callable( ) schema.function_lists.append( - FunctionDescription(name.upper(), parameters, return_type, aggregation) + FunctionDescription( + name.upper(), sql_parameters, sql_return_type, aggregation + ) ) schema.function_lists.append( - FunctionDescription(name.lower(), parameters, return_type, aggregation) + FunctionDescription( + name.lower(), sql_parameters, sql_return_type, aggregation + ) ) schema.functions[lower_name] = f diff --git a/dask_sql/datacontainer.py b/dask_sql/datacontainer.py index db77c9dfc..f81952e68 100644 --- a/dask_sql/datacontainer.py +++ b/dask_sql/datacontainer.py @@ -198,11 +198,6 @@ def __init__(self, func, row_udf: bool, params, return_type=None): self.names = [param[0] for param in params] - if return_type is None: - # These UDFs go through apply and without providing - # a return type, dask will attempt to guess it, and - # dask might be wrong. - raise ValueError("Return type must be provided") self.meta = (None, return_type) def __call__(self, *args, **kwargs): @@ -218,7 +213,6 @@ def __call__(self, *args, **kwargs): df = column_args[0].to_frame(self.names[0]) for name, col in zip(self.names[1:], column_args[1:]): df[name] = col - result = df.apply( self.func, axis=1, args=tuple(scalar_args), meta=self.meta ).astype(self.meta[1]) diff --git a/dask_sql/mappings.py b/dask_sql/mappings.py index e59025918..623c38a37 100644 --- a/dask_sql/mappings.py +++ b/dask_sql/mappings.py @@ -88,6 +88,11 @@ def python_to_sql_type(python_type): """Mapping between python and SQL types.""" + if python_type in (int, float): + python_type = np.dtype(python_type) + elif python_type is str: + python_type = np.dtype("object") + if isinstance(python_type, np.dtype): python_type = python_type.type diff --git a/tests/integration/test_function.py b/tests/integration/test_function.py index 92fc58b14..d8ba40c0f 100644 --- a/tests/integration/test_function.py +++ b/tests/integration/test_function.py @@ -52,17 +52,12 @@ def f(row): @pytest.mark.parametrize( "retty", - [None, np.float64, np.float32, np.int64, np.int32, np.int16, np.int8, np.bool_], + [np.float64, np.float32, np.int64, np.int32, np.int16, np.int8, np.bool_], ) def test_custom_function_row_return_types(c, df, retty): def f(row): return row["x"] ** 2 - if retty is None: - with pytest.raises(ValueError): - c.register_function(f, "f", [("x", np.float64)], retty, row_udf=True) - return - c.register_function(f, "f", [("x", np.float64)], retty, row_udf=True) return_df = c.sql("SELECT F(a) AS a FROM df") @@ -199,3 +194,17 @@ def f(x): c.register_aggregation(fagg, "fagg", [("x", np.float64)], np.float64) c.register_aggregation(fagg, "fagg", [("x", np.float64)], np.float64, replace=True) + + +@pytest.mark.parametrize("dtype", [np.timedelta64, None, "a string"]) +def test_unsupported_dtype(c, dtype): + def f(x): + return x**2 + + # test that an invalid return type raises + with pytest.raises(NotImplementedError): + c.register_function(f, "f", [("x", np.int64)], dtype) + + # test that an invalid param type raises + with pytest.raises(NotImplementedError): + c.register_function(f, "f", [("x", dtype)], np.int64) diff --git a/tests/unit/test_context.py b/tests/unit/test_context.py index b8cfa6504..fca5c7454 100644 --- a/tests/unit/test_context.py +++ b/tests/unit/test_context.py @@ -6,6 +6,7 @@ from dask_sql import Context from dask_sql.datacontainer import Statistics +from dask_sql.mappings import python_to_sql_type from tests.utils import assert_eq try: @@ -198,6 +199,11 @@ def g(gpu=gpu): g(gpu=gpu) +int_sql_type = python_to_sql_type(int) +float_sql_type = python_to_sql_type(float) +str_sql_type = python_to_sql_type(str) + + def test_function_adding(): c = Context() @@ -211,12 +217,12 @@ def test_function_adding(): assert c.schema[c.schema_name].functions["f"].func == f assert len(c.schema[c.schema_name].function_lists) == 2 assert c.schema[c.schema_name].function_lists[0].name == "F" - assert c.schema[c.schema_name].function_lists[0].parameters == [("x", int)] - assert c.schema[c.schema_name].function_lists[0].return_type == float + assert c.schema[c.schema_name].function_lists[0].parameters == [("x", int_sql_type)] + assert c.schema[c.schema_name].function_lists[0].return_type == float_sql_type assert not c.schema[c.schema_name].function_lists[0].aggregation assert c.schema[c.schema_name].function_lists[1].name == "f" - assert c.schema[c.schema_name].function_lists[1].parameters == [("x", int)] - assert c.schema[c.schema_name].function_lists[1].return_type == float + assert c.schema[c.schema_name].function_lists[1].parameters == [("x", int_sql_type)] + assert c.schema[c.schema_name].function_lists[1].return_type == float_sql_type assert not c.schema[c.schema_name].function_lists[1].aggregation # Without replacement @@ -226,12 +232,16 @@ def test_function_adding(): assert c.schema[c.schema_name].functions["f"].func == f assert len(c.schema[c.schema_name].function_lists) == 4 assert c.schema[c.schema_name].function_lists[2].name == "F" - assert c.schema[c.schema_name].function_lists[2].parameters == [("x", float)] - assert c.schema[c.schema_name].function_lists[2].return_type == int + assert c.schema[c.schema_name].function_lists[2].parameters == [ + ("x", float_sql_type) + ] + assert c.schema[c.schema_name].function_lists[2].return_type == int_sql_type assert not c.schema[c.schema_name].function_lists[2].aggregation assert c.schema[c.schema_name].function_lists[3].name == "f" - assert c.schema[c.schema_name].function_lists[3].parameters == [("x", float)] - assert c.schema[c.schema_name].function_lists[3].return_type == int + assert c.schema[c.schema_name].function_lists[3].parameters == [ + ("x", float_sql_type) + ] + assert c.schema[c.schema_name].function_lists[3].return_type == int_sql_type assert not c.schema[c.schema_name].function_lists[3].aggregation # With replacement @@ -242,12 +252,12 @@ def test_function_adding(): assert c.schema[c.schema_name].functions["f"].func == f assert len(c.schema[c.schema_name].function_lists) == 2 assert c.schema[c.schema_name].function_lists[0].name == "F" - assert c.schema[c.schema_name].function_lists[0].parameters == [("x", str)] - assert c.schema[c.schema_name].function_lists[0].return_type == str + assert c.schema[c.schema_name].function_lists[0].parameters == [("x", str_sql_type)] + assert c.schema[c.schema_name].function_lists[0].return_type == str_sql_type assert not c.schema[c.schema_name].function_lists[0].aggregation assert c.schema[c.schema_name].function_lists[1].name == "f" - assert c.schema[c.schema_name].function_lists[1].parameters == [("x", str)] - assert c.schema[c.schema_name].function_lists[1].return_type == str + assert c.schema[c.schema_name].function_lists[1].parameters == [("x", str_sql_type)] + assert c.schema[c.schema_name].function_lists[1].return_type == str_sql_type assert not c.schema[c.schema_name].function_lists[1].aggregation @@ -264,12 +274,12 @@ def test_aggregation_adding(): assert c.schema[c.schema_name].functions["f"] == f assert len(c.schema[c.schema_name].function_lists) == 2 assert c.schema[c.schema_name].function_lists[0].name == "F" - assert c.schema[c.schema_name].function_lists[0].parameters == [("x", int)] - assert c.schema[c.schema_name].function_lists[0].return_type == float + assert c.schema[c.schema_name].function_lists[0].parameters == [("x", int_sql_type)] + assert c.schema[c.schema_name].function_lists[0].return_type == float_sql_type assert c.schema[c.schema_name].function_lists[0].aggregation assert c.schema[c.schema_name].function_lists[1].name == "f" - assert c.schema[c.schema_name].function_lists[1].parameters == [("x", int)] - assert c.schema[c.schema_name].function_lists[1].return_type == float + assert c.schema[c.schema_name].function_lists[1].parameters == [("x", int_sql_type)] + assert c.schema[c.schema_name].function_lists[1].return_type == float_sql_type assert c.schema[c.schema_name].function_lists[1].aggregation # Without replacement @@ -279,12 +289,16 @@ def test_aggregation_adding(): assert c.schema[c.schema_name].functions["f"] == f assert len(c.schema[c.schema_name].function_lists) == 4 assert c.schema[c.schema_name].function_lists[2].name == "F" - assert c.schema[c.schema_name].function_lists[2].parameters == [("x", float)] - assert c.schema[c.schema_name].function_lists[2].return_type == int + assert c.schema[c.schema_name].function_lists[2].parameters == [ + ("x", float_sql_type) + ] + assert c.schema[c.schema_name].function_lists[2].return_type == int_sql_type assert c.schema[c.schema_name].function_lists[2].aggregation assert c.schema[c.schema_name].function_lists[3].name == "f" - assert c.schema[c.schema_name].function_lists[3].parameters == [("x", float)] - assert c.schema[c.schema_name].function_lists[3].return_type == int + assert c.schema[c.schema_name].function_lists[3].parameters == [ + ("x", float_sql_type) + ] + assert c.schema[c.schema_name].function_lists[3].return_type == int_sql_type assert c.schema[c.schema_name].function_lists[3].aggregation # With replacement @@ -295,12 +309,12 @@ def test_aggregation_adding(): assert c.schema[c.schema_name].functions["f"] == f assert len(c.schema[c.schema_name].function_lists) == 2 assert c.schema[c.schema_name].function_lists[0].name == "F" - assert c.schema[c.schema_name].function_lists[0].parameters == [("x", str)] - assert c.schema[c.schema_name].function_lists[0].return_type == str + assert c.schema[c.schema_name].function_lists[0].parameters == [("x", str_sql_type)] + assert c.schema[c.schema_name].function_lists[0].return_type == str_sql_type assert c.schema[c.schema_name].function_lists[0].aggregation assert c.schema[c.schema_name].function_lists[1].name == "f" - assert c.schema[c.schema_name].function_lists[1].parameters == [("x", str)] - assert c.schema[c.schema_name].function_lists[1].return_type == str + assert c.schema[c.schema_name].function_lists[1].parameters == [("x", str_sql_type)] + assert c.schema[c.schema_name].function_lists[1].return_type == str_sql_type assert c.schema[c.schema_name].function_lists[1].aggregation From 712a2af4bbe9b71b5e0f26d09a65bbfc7dfe4615 Mon Sep 17 00:00:00 2001 From: Han Wang Date: Tue, 26 Jul 2022 11:06:17 -0700 Subject: [PATCH 37/44] Set Dask-sql as the default Fugue Dask engine when installed (#640) * Set Dask-sql as the default Fugue Dask engine when installed * Set Dask-sql as the default Fugue Dask engine when installed * Add max version constraint for `fugue` (#639) * Remove antlr4-python3-runtime constraint from 3.9+ test envs * Revert "Remove antlr4-python3-runtime constraint from 3.9+ test envs" This reverts commit ef3065658b669bfbef41fc71cf6b7369b443b362. * Add max version constraint for fugue in 3.9+ envs * Constrain Fugue in remaining env/setup files * Clarify fugue constraint comments * Add pinning back to python 3.8 jdk11 tests * More reversions to python 3.8 jdk11 testing env * update * update * update * fix tests * update tests * update a few things * update * fix * conda install fugue in testing envs * Remove diff from features notebook * Alter documentation to mention automatic registration of execution engine * Expand FugueSQL notebook * Don't manually close client in simple statement test Co-authored-by: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> --- .../environment-3.10-jdk11-dev.yaml | 12 +- .../environment-3.10-jdk8-dev.yaml | 12 +- .../environment-3.8-jdk11-dev.yaml | 11 +- .../environment-3.8-jdk8-dev.yaml | 11 +- .../environment-3.9-jdk11-dev.yaml | 12 +- .../environment-3.9-jdk8-dev.yaml | 12 +- dask_sql/integrations/fugue.py | 24 +- docs/environment.yml | 3 +- docs/requirements-docs.txt | 3 +- docs/source/fugue.rst | 2 +- notebooks/FugueSQL.ipynb | 577 ++++++++++++++++++ setup.py | 8 +- tests/integration/test_fugue.py | 13 +- 13 files changed, 623 insertions(+), 77 deletions(-) create mode 100644 notebooks/FugueSQL.ipynb diff --git a/continuous_integration/environment-3.10-jdk11-dev.yaml b/continuous_integration/environment-3.10-jdk11-dev.yaml index d6c45735e..59033ae3c 100644 --- a/continuous_integration/environment-3.10-jdk11-dev.yaml +++ b/continuous_integration/environment-3.10-jdk11-dev.yaml @@ -6,6 +6,7 @@ dependencies: - dask-ml>=2022.1.22 - dask>=2022.3.0 - fastapi>=0.69.0 +- fugue>=0.7.0 - intake>=0.6.0 - jpype1>=1.0.2 - jsonschema @@ -31,14 +32,3 @@ dependencies: - tpot - tzlocal>=2.1 - uvicorn>=0.11.3 -# fugue dependencies; remove when we conda install fugue -- adagio -- antlr4-python3-runtime<4.10 -- ciso8601 -- fs -- pip -- qpd -- triad -- pip: - # FIXME: tests are failing with fugue 0.7.0 - - fugue[sql]>=0.5.3,<0.7.0 diff --git a/continuous_integration/environment-3.10-jdk8-dev.yaml b/continuous_integration/environment-3.10-jdk8-dev.yaml index f28206297..742e4ba1d 100644 --- a/continuous_integration/environment-3.10-jdk8-dev.yaml +++ b/continuous_integration/environment-3.10-jdk8-dev.yaml @@ -6,6 +6,7 @@ dependencies: - dask-ml>=2022.1.22 - dask>=2022.3.0 - fastapi>=0.69.0 +- fugue>=0.7.0 - intake>=0.6.0 - jpype1>=1.0.2 - jsonschema @@ -31,14 +32,3 @@ dependencies: - tpot - tzlocal>=2.1 - uvicorn>=0.11.3 -# fugue dependencies; remove when we conda install fugue -- adagio -- antlr4-python3-runtime<4.10 -- ciso8601 -- fs -- pip -- qpd -- triad -- pip: - # FIXME: tests are failing with fugue 0.7.0 - - fugue[sql]>=0.5.3,<0.7.0 diff --git a/continuous_integration/environment-3.8-jdk11-dev.yaml b/continuous_integration/environment-3.8-jdk11-dev.yaml index b865bb82f..17e39266a 100644 --- a/continuous_integration/environment-3.8-jdk11-dev.yaml +++ b/continuous_integration/environment-3.8-jdk11-dev.yaml @@ -6,6 +6,7 @@ dependencies: - dask-ml=2022.1.22 - dask=2022.3.0 - fastapi=0.69.0 +- fugue=0.7.0 - intake=0.6.0 - jpype1=1.0.2 - jsonschema @@ -31,13 +32,3 @@ dependencies: - tpot - tzlocal=2.1 - uvicorn=0.11.3 -# fugue dependencies; remove when we conda install fugue -- adagio -- antlr4-python3-runtime<4.10 -- ciso8601 -- fs -- pip -- qpd -- triad -- pip: - - fugue[sql]==0.5.3 diff --git a/continuous_integration/environment-3.8-jdk8-dev.yaml b/continuous_integration/environment-3.8-jdk8-dev.yaml index b143fcc6c..b6cbf582c 100644 --- a/continuous_integration/environment-3.8-jdk8-dev.yaml +++ b/continuous_integration/environment-3.8-jdk8-dev.yaml @@ -6,6 +6,7 @@ dependencies: - dask-ml=2022.1.22 - dask=2022.3.0 - fastapi=0.69.0 +- fugue=0.7.0 - intake=0.6.0 - jpype1=1.0.2 - jsonschema @@ -31,13 +32,3 @@ dependencies: - tpot - tzlocal=2.1 - uvicorn=0.11.3 -# fugue dependencies; remove when we conda install fugue -- adagio -- antlr4-python3-runtime<4.10 -- ciso8601 -- fs -- pip -- qpd -- triad -- pip: - - fugue[sql]==0.5.3 diff --git a/continuous_integration/environment-3.9-jdk11-dev.yaml b/continuous_integration/environment-3.9-jdk11-dev.yaml index 5a1c19777..f38cafade 100644 --- a/continuous_integration/environment-3.9-jdk11-dev.yaml +++ b/continuous_integration/environment-3.9-jdk11-dev.yaml @@ -6,6 +6,7 @@ dependencies: - dask-ml>=2022.1.22 - dask>=2022.3.0 - fastapi>=0.69.0 +- fugue>=0.7.0 - intake>=0.6.0 - jpype1>=1.0.2 - jsonschema @@ -31,14 +32,3 @@ dependencies: - tpot - tzlocal>=2.1 - uvicorn>=0.11.3 -# fugue dependencies; remove when we conda install fugue -- adagio -- antlr4-python3-runtime<4.10 -- ciso8601 -- fs -- pip -- qpd -- triad -- pip: - # FIXME: tests are failing with fugue 0.7.0 - - fugue[sql]>=0.5.3,<0.7.0 diff --git a/continuous_integration/environment-3.9-jdk8-dev.yaml b/continuous_integration/environment-3.9-jdk8-dev.yaml index 6f8367e51..c16d95f01 100644 --- a/continuous_integration/environment-3.9-jdk8-dev.yaml +++ b/continuous_integration/environment-3.9-jdk8-dev.yaml @@ -6,6 +6,7 @@ dependencies: - dask-ml>=2022.1.22 - dask>=2022.3.0 - fastapi>=0.69.0 +- fugue>=0.7.0 - intake>=0.6.0 - jpype1>=1.0.2 - jsonschema @@ -31,14 +32,3 @@ dependencies: - tpot - tzlocal>=2.1 - uvicorn>=0.11.3 -# fugue dependencies; remove when we conda install fugue -- adagio -- antlr4-python3-runtime<4.10 -- ciso8601 -- fs -- pip -- qpd -- triad -- pip: - # FIXME: tests are failing with fugue 0.7.0 - - fugue[sql]>=0.5.3,<0.7.0 diff --git a/dask_sql/integrations/fugue.py b/dask_sql/integrations/fugue.py index ce685a1ee..11b0c4d3a 100644 --- a/dask_sql/integrations/fugue.py +++ b/dask_sql/integrations/fugue.py @@ -1,8 +1,10 @@ try: import fugue import fugue_dask + from dask.distributed import Client from fugue import WorkflowDataFrame, register_execution_engine from fugue_sql import FugueSQLWorkflow + from triad import run_at_def from triad.utils.convert import get_caller_global_local_vars except ImportError: # pragma: no cover raise ImportError( @@ -15,9 +17,25 @@ from dask_sql.context import Context -register_execution_engine( - "dask", lambda conf: DaskSQLExecutionEngine(conf), on_dup="overwrite" -) + +@run_at_def +def _register_engines() -> None: + """Register (overwrite) the default Dask execution engine of Fugue. This + function is invoked as an entrypoint, users don't need to call it explicitly. + """ + register_execution_engine( + "dask", + lambda conf, **kwargs: DaskSQLExecutionEngine(conf=conf), + on_dup="overwrite", + ) + + register_execution_engine( + Client, + lambda engine, conf, **kwargs: DaskSQLExecutionEngine( + dask_client=engine, conf=conf + ), + on_dup="overwrite", + ) class DaskSQLEngine(fugue.execution.execution_engine.SQLEngine): diff --git a/docs/environment.yml b/docs/environment.yml index f8e0df2f8..5704e7e29 100644 --- a/docs/environment.yml +++ b/docs/environment.yml @@ -10,8 +10,7 @@ dependencies: - maven>=3.6.0 - dask>=2022.3.0 - pandas>=1.1.2 - # FIXME: tests are failing with fugue 0.7.0 - - fugue>=0.5.3,<0.7.0 + - fugue>=0.7.0 - jpype1>=1.0.2 - fastapi>=0.69.0 - uvicorn>=0.11.3 diff --git a/docs/requirements-docs.txt b/docs/requirements-docs.txt index 09870d22e..be37d0b83 100644 --- a/docs/requirements-docs.txt +++ b/docs/requirements-docs.txt @@ -3,8 +3,7 @@ sphinx-tabs dask-sphinx-theme>=3.0.0 dask>=2022.3.0 pandas>=1.1.2 -# FIXME: tests are failing with fugue 0.7.0 -fugue>=0.5.3,<0.7.0 +fugue>=0.7.0 jpype1>=1.0.2 fastapi>=0.69.0 uvicorn>=0.11.3 diff --git a/docs/source/fugue.rst b/docs/source/fugue.rst index 264d19fcd..972229c13 100644 --- a/docs/source/fugue.rst +++ b/docs/source/fugue.rst @@ -8,7 +8,7 @@ In order to offer a "best of both worlds" solution, dask-sql includes several op dask-sql as a FugueSQL engine ----------------------------- -FugueSQL users unfamiliar with dask-sql can take advantage of its functionality with minimal code changes by passing :class:`dask_sql.integrations.fugue.DaskSQLExecutionEngine` into the ``FugueSQLWorkflow`` being used to execute commands. +FugueSQL users unfamiliar with dask-sql can take advantage of its functionality by installing it in an environment alongside Fugue; this will automatically register :class:`dask_sql.integrations.fugue.DaskSQLExecutionEngine` as the default Dask execution engine for FugueSQL queries. For more information and sample usage, see `Fugue — dask-sql as a FugueSQL engine `_. Using FugueSQL on an existing ``Context`` diff --git a/notebooks/FugueSQL.ipynb b/notebooks/FugueSQL.ipynb new file mode 100644 index 000000000..1d59b8f78 --- /dev/null +++ b/notebooks/FugueSQL.ipynb @@ -0,0 +1,577 @@ +{ + "cells": [ + { + "cell_type": "markdown", + "id": "f39e2dbc-21a1-4d9a-bed7-e2bf2bd25bb8", + "metadata": {}, + "source": [ + "# FugueSQL Integrations\n", + "\n", + "[FugueSQL](https://fugue-tutorials.readthedocs.io/tutorials/fugue_sql/index.html) is a related project that aims to provide a unified SQL interface for a variety of different computing frameworks, including Dask.\n", + "While it offers a SQL engine with a larger set of supported commands, this comes at the cost of slower performance when using Dask in comparison to dask-sql.\n", + "In order to offer a \"best of both worlds\" solution, dask-sql can easily be integrated with FugueSQL, using its faster implementation of SQL commands when possible and falling back on FugueSQL's implementation when necessary." + ] + }, + { + "cell_type": "markdown", + "id": "90e31400", + "metadata": {}, + "source": [ + "## Setup\n", + "\n", + "FugueSQL offers the cell magic `%%fsql`, which can be used to define and execute queries entirely in SQL, with no need for external Python code!\n", + "\n", + "To use this cell magic, users must install [fugue-jupyter](https://pypi.org/project/fugue-jupyter/), which will additionally provide SQL syntax highlighting (note that the kernel must be restart after installing):" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "96c3ad1a", + "metadata": {}, + "outputs": [], + "source": [ + "!pip install fugue-jupyter" + ] + }, + { + "cell_type": "markdown", + "id": "ae79361a", + "metadata": {}, + "source": [ + "And run `fugue_jupyter.setup()` to register the magic:" + ] + }, + { + "cell_type": "code", + "execution_count": 1, + "id": "2df05f5b", + "metadata": {}, + "outputs": [], + "source": [ + "from fugue_jupyter import setup\n", + "\n", + "setup()" + ] + }, + { + "cell_type": "markdown", + "id": "d3b8bfe5", + "metadata": {}, + "source": [ + "We will also start up a Dask client, which can be specified as an execution engine for FugueSQL queries:" + ] + }, + { + "cell_type": "code", + "execution_count": 2, + "id": "a35d98e6-f24e-46c4-a4e6-b64d649d8ba7", + "metadata": {}, + "outputs": [], + "source": [ + "from dask.distributed import Client\n", + "\n", + "client = Client()" + ] + }, + { + "cell_type": "markdown", + "id": "bcb96523", + "metadata": {}, + "source": [ + "## dask-sql as a FugueSQL execution engine\n", + "\n", + "When dask-sql is installed, its `DaskSQLExecutionEngine` is automatically registered as the default engine for FugueSQL queries ran on Dask.\n", + "We can then use it to run queries with the `%%fsql` cell magic, specifying `dask` as the execution engine to run the query on:" + ] + }, + { + "cell_type": "code", + "execution_count": 3, + "id": "ff633572-ad08-4de1-8678-a8fbd09effd1", + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
a
0xyz
\n", + "
" + ], + "text/plain": [ + " a\n", + "0 xyz" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "schema: a:str" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "%%fsql dask\n", + "\n", + "CREATE [[\"xyz\"], [\"xxx\"]] SCHEMA a:str\n", + "SELECT * WHERE a LIKE '%y%'\n", + "PRINT" + ] + }, + { + "cell_type": "markdown", + "id": "7f16b7d9-6b45-4caf-bbcb-63cc5d858556", + "metadata": {}, + "source": [ + "We can also use the `YIELD` keyword to register the results of our queries into Python objects:" + ] + }, + { + "cell_type": "code", + "execution_count": 4, + "id": "521965bc-1a4c-49ab-b48f-789351cb24d4", + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
b
0xyz
1xxx-
\n", + "
" + ], + "text/plain": [ + " b\n", + "0 xyz\n", + "1 xxx-" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "schema: b:str" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "%%fsql dask\n", + "src = CREATE [[\"xyz\"], [\"xxx\"]] SCHEMA a:str\n", + "\n", + "a = SELECT a AS b WHERE a LIKE '%y%'\n", + " YIELD DATAFRAME AS test\n", + "\n", + "b = SELECT CONCAT(a, '-') AS b FROM src WHERE a LIKE '%xx%'\n", + " YIELD DATAFRAME AS test1\n", + "\n", + "SELECT * FROM a UNION SELECT * FROM b\n", + "PRINT" + ] + }, + { + "cell_type": "markdown", + "id": "dfbb0a9a", + "metadata": {}, + "source": [ + "Which can then be interacted with outside of SQL:" + ] + }, + { + "cell_type": "code", + "execution_count": 5, + "id": "79a3e87a-2764-410c-b257-c710c4a6c6d4", + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
Dask DataFrame Structure:
\n", + "
\n", + "\n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
b
npartitions=2
object
...
...
\n", + "
\n", + "
Dask Name: rename, 16 tasks
" + ], + "text/plain": [ + "Dask DataFrame Structure:\n", + " b\n", + "npartitions=2 \n", + " object\n", + " ...\n", + " ...\n", + "Dask Name: rename, 16 tasks" + ] + }, + "execution_count": 5, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "test.native # a Dask DataFrame" + ] + }, + { + "cell_type": "code", + "execution_count": 6, + "id": "c98cb652-06e2-444a-b70a-fdd3de9ecd15", + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
b
1xxx-
\n", + "
" + ], + "text/plain": [ + " b\n", + "1 xxx-" + ] + }, + "execution_count": 6, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "test1.native.compute()" + ] + }, + { + "cell_type": "markdown", + "id": "932ede31-90b2-49e5-9f4d-7cf1b8d919d2", + "metadata": {}, + "source": [ + "We can also run the equivalent of these queries in python code using `fugue_sql.fsql`, passing the Dask client into its `run` method to specify Dask as an execution engine:" + ] + }, + { + "cell_type": "code", + "execution_count": 7, + "id": "c265b170-de4d-4fab-aeae-9f94031e960d", + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
a
0xyz
\n", + "
" + ], + "text/plain": [ + " a\n", + "0 xyz" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "schema: a:str" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/plain": [ + "DataFrames()" + ] + }, + "execution_count": 7, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "from fugue_sql import fsql\n", + "\n", + "fsql(\"\"\"\n", + "CREATE [[\"xyz\"], [\"xxx\"]] SCHEMA a:str\n", + "SELECT * WHERE a LIKE '%y%'\n", + "PRINT\n", + "\"\"\").run(client)" + ] + }, + { + "cell_type": "code", + "execution_count": 8, + "id": "77e3bf50-8c8b-4e2f-a5e7-28b1d86499d7", + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
Dask DataFrame Structure:
\n", + "
\n", + "\n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
a
npartitions=2
object
...
...
\n", + "
\n", + "
Dask Name: rename, 16 tasks
" + ], + "text/plain": [ + "Dask DataFrame Structure:\n", + " a\n", + "npartitions=2 \n", + " object\n", + " ...\n", + " ...\n", + "Dask Name: rename, 16 tasks" + ] + }, + "execution_count": 8, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "result = fsql(\"\"\"\n", + "CREATE [[\"xyz\"], [\"xxx\"]] SCHEMA a:str\n", + "SELECT * WHERE a LIKE '%y%'\n", + "YIELD DATAFRAME AS test2\n", + "\"\"\").run(client)\n", + "\n", + "result[\"test2\"].native # a Dask DataFrame" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "7d4c71d4-238f-4c72-8609-dbbe0782aea9", + "metadata": {}, + "outputs": [], + "source": [] + } + ], + "metadata": { + "kernelspec": { + "display_name": "Python 3 (ipykernel)", + "language": "python", + "name": "python3" + }, + "language_info": { + "codemirror_mode": { + "name": "ipython", + "version": 3 + }, + "file_extension": ".py", + "mimetype": "text/x-python", + "name": "python", + "nbconvert_exporter": "python", + "pygments_lexer": "ipython3", + "version": "3.9.13" + }, + "vscode": { + "interpreter": { + "hash": "656801d214ad98d4b301386b078628ce3ae2dbd81a59ed4deed7a5b13edfab09" + } + } + }, + "nbformat": 4, + "nbformat_minor": 5 +} diff --git a/setup.py b/setup.py index 79f2247b3..1d52789d1 100755 --- a/setup.py +++ b/setup.py @@ -112,14 +112,16 @@ def build(self): "black==22.3.0", "isort==5.7.0", ], - # FIXME: tests are failing with fugue 0.7.0 - "fugue": ["fugue[sql]>=0.5.3,<0.7.0"], + "fugue": ["fugue>=0.7.0"], }, entry_points={ "console_scripts": [ "dask-sql-server = dask_sql.server.app:main", "dask-sql = dask_sql.cmd:main", - ] + ], + "fugue.plugins": [ + "dasksql = dask_sql.integrations.fugue:_register_engines[fugue]" + ], }, zip_safe=False, cmdclass=cmdclass, diff --git a/tests/integration/test_fugue.py b/tests/integration/test_fugue.py index 951bf7a48..d846b5559 100644 --- a/tests/integration/test_fugue.py +++ b/tests/integration/test_fugue.py @@ -13,7 +13,8 @@ from dask_sql.integrations.fugue import DaskSQLExecutionEngine, fsql_dask -def test_simple_statement(): +@skip_if_external_scheduler +def test_simple_statement(client): with fugue_sql.FugueSQLWorkflow(DaskSQLExecutionEngine) as dag: df = dag.df([[0, "hello"], [1, "world"]], "a:int64,b:str") dag("SELECT * FROM df WHERE a > 0 YIELD DATAFRAME AS result") @@ -32,11 +33,19 @@ def test_simple_statement(): return_df = result["result"].as_pandas() assert_eq(return_df, pd.DataFrame({"a": [1], "b": ["world"]})) + result = fugue_sql.fsql( + "SELECT * FROM df WHERE a > 0 YIELD DATAFRAME AS result", + df=pdf, + ).run(client) + + return_df = result["result"].as_pandas() + assert_eq(return_df, pd.DataFrame({"a": [1], "b": ["world"]})) + # TODO: Revisit fixing this on an independant cluster (without dask-sql) based on the # discussion in https://github.com/dask-contrib/dask-sql/issues/407 @skip_if_external_scheduler -def test_fsql(): +def test_fsql(client): def assert_fsql(df: pd.DataFrame) -> None: assert_eq(df, pd.DataFrame({"a": [1]})) From a9be03a61cbb06e404aa2f0ef5ada7ac082ce018 Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Mon, 1 Aug 2022 10:57:24 -0700 Subject: [PATCH 38/44] Add Rust setup to upstream testing workflow --- .github/workflows/test-upstream.yml | 27 +++++++++++++++++++++++++++ 1 file changed, 27 insertions(+) diff --git a/.github/workflows/test-upstream.yml b/.github/workflows/test-upstream.yml index 31d102b4c..fd0cae327 100644 --- a/.github/workflows/test-upstream.yml +++ b/.github/workflows/test-upstream.yml @@ -38,6 +38,15 @@ jobs: channels: dask/label/dev,conda-forge,nodefaults activate-environment: dask-sql environment-file: ${{ env.CONDA_FILE }} + - name: Setup Rust Toolchain + uses: actions-rs/toolchain@v1 + id: rust-toolchain + with: + toolchain: stable + override: true + - name: Build the Rust DataFusion bindings + run: | + python setup.py build install - name: Install hive testing dependencies for Linux if: matrix.os == 'ubuntu-latest' run: | @@ -68,6 +77,15 @@ jobs: channels: dask/label/dev,conda-forge,nodefaults activate-environment: dask-sql environment-file: continuous_integration/environment-3.9-dev.yaml + - name: Setup Rust Toolchain + uses: actions-rs/toolchain@v1 + id: rust-toolchain + with: + toolchain: stable + override: true + - name: Build the Rust DataFusion bindings + run: | + python setup.py build install - name: Install cluster dependencies run: | mamba install python-blosc lz4 -c conda-forge @@ -105,6 +123,15 @@ jobs: mamba-version: "*" channels: dask/label/dev,conda-forge,nodefaults channel-priority: strict + - name: Setup Rust Toolchain + uses: actions-rs/toolchain@v1 + id: rust-toolchain + with: + toolchain: stable + override: true + - name: Build the Rust DataFusion bindings + run: | + python setup.py build install - name: Install upstream dev Dask / dask-ml if: needs.detect-ci-trigger.outputs.triggered == 'true' run: | From 138e70e54ab59ee93a82b368d1516c31baa6ae7f Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Mon, 1 Aug 2022 11:03:38 -0700 Subject: [PATCH 39/44] Resolve style failures --- dask_sql/physical/rex/core/call.py | 14 -------------- tests/unit/test_context.py | 1 + 2 files changed, 1 insertion(+), 14 deletions(-) diff --git a/dask_sql/physical/rex/core/call.py b/dask_sql/physical/rex/core/call.py index 543baf7ff..79a2d099f 100644 --- a/dask_sql/physical/rex/core/call.py +++ b/dask_sql/physical/rex/core/call.py @@ -16,7 +16,6 @@ from dask_planner.rust import SqlTypeName from dask_sql.datacontainer import DataContainer -from dask_sql.java import get_java_class from dask_sql.mappings import cast_column_to_type, sql_to_python_type from dask_sql.physical.rex import RexConverter from dask_sql.physical.rex.base import BaseRexPlugin @@ -983,16 +982,3 @@ def convert( return operation(*operands, **kwargs) # TODO: We have information on the typing here - we should use it - - -def check_special_operator(operator: "org.apache.calcite.sql.fun"): - """ - Check for special operator classes that have an overloaded name with other - operator type/kinds. - - eg: sqlDatetimeSubtractionOperator has the sqltype and kind of the `-` or `minus` operation. - """ - special_op_to_name = { - "org.apache.calcite.sql.fun.SqlDatetimeSubtractionOperator": "datetime_subtraction" - } - return special_op_to_name.get(get_java_class(operator), None) diff --git a/tests/unit/test_context.py b/tests/unit/test_context.py index ca6a98602..f10025e5a 100644 --- a/tests/unit/test_context.py +++ b/tests/unit/test_context.py @@ -5,6 +5,7 @@ import pytest from dask_sql import Context +from dask_sql.mappings import python_to_sql_type from tests.utils import assert_eq try: From a0c6344f19889bf9acb37fd0cc7e3d22a6c11083 Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Mon, 1 Aug 2022 11:19:09 -0700 Subject: [PATCH 40/44] Bump fugue version in CI envs --- continuous_integration/environment-3.10-dev.yaml | 11 +---------- continuous_integration/environment-3.8-dev.yaml | 11 +---------- continuous_integration/environment-3.9-dev.yaml | 11 +---------- 3 files changed, 3 insertions(+), 30 deletions(-) diff --git a/continuous_integration/environment-3.10-dev.yaml b/continuous_integration/environment-3.10-dev.yaml index 7f5eb34a3..1c5ff1a16 100644 --- a/continuous_integration/environment-3.10-dev.yaml +++ b/continuous_integration/environment-3.10-dev.yaml @@ -6,6 +6,7 @@ dependencies: - dask-ml>=2022.1.22 - dask>=2022.3.0 - fastapi>=0.69.0 +- fugue>=0.7.0 - intake>=0.6.0 - jsonschema - lightgbm @@ -31,13 +32,3 @@ dependencies: - tpot - tzlocal>=2.1 - uvicorn>=0.11.3 -# fugue dependencies; remove when we conda install fugue -- adagio -- antlr4-python3-runtime<4.10 -- ciso8601 -- fs -- pip -- qpd -- triad -- pip: - - fugue[sql]>=0.5.3 diff --git a/continuous_integration/environment-3.8-dev.yaml b/continuous_integration/environment-3.8-dev.yaml index 1506c7f38..c5633f17d 100644 --- a/continuous_integration/environment-3.8-dev.yaml +++ b/continuous_integration/environment-3.8-dev.yaml @@ -6,6 +6,7 @@ dependencies: - dask-ml=2022.1.22 - dask=2022.3.0 - fastapi=0.69.0 +- fugue=0.7.0 - intake=0.6.0 - jsonschema - lightgbm @@ -31,13 +32,3 @@ dependencies: - tpot - tzlocal=2.1 - uvicorn=0.11.3 -# fugue dependencies; remove when we conda install fugue -- adagio -- antlr4-python3-runtime<4.10 -- ciso8601 -- fs -- pip -- qpd -- triad -- pip: - - fugue[sql]==0.5.3 diff --git a/continuous_integration/environment-3.9-dev.yaml b/continuous_integration/environment-3.9-dev.yaml index 7127c827f..8e82e80e7 100644 --- a/continuous_integration/environment-3.9-dev.yaml +++ b/continuous_integration/environment-3.9-dev.yaml @@ -6,6 +6,7 @@ dependencies: - dask-ml>=2022.1.22 - dask>=2022.3.0 - fastapi>=0.69.0 +- fugue>=0.7.0 - intake>=0.6.0 - jsonschema - lightgbm @@ -31,13 +32,3 @@ dependencies: - tpot - tzlocal>=2.1 - uvicorn>=0.11.3 -# fugue dependencies; remove when we conda install fugue -- adagio -- antlr4-python3-runtime<4.10 -- ciso8601 -- fs -- pip -- qpd -- triad -- pip: - - fugue[sql]>=0.5.3 From 73f5228ba5a7cfdcf1f055901715286e0f9770c7 Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Mon, 1 Aug 2022 11:32:03 -0700 Subject: [PATCH 41/44] Add back scalar case for cast operation --- dask_sql/physical/rex/core/call.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dask_sql/physical/rex/core/call.py b/dask_sql/physical/rex/core/call.py index 79a2d099f..77430b562 100644 --- a/dask_sql/physical/rex/core/call.py +++ b/dask_sql/physical/rex/core/call.py @@ -238,6 +238,9 @@ def __init__(self): super().__init__(self.cast) def cast(self, operand, rex=None) -> SeriesOrScalar: + if not is_frame(operand): # pragma: no cover + return operand + output_type = str(rex.getType()) python_type = sql_to_python_type(SqlTypeName.fromString(output_type.upper())) From 71403fe6e8ef9425edf1b878960211b1e92b7dcc Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Mon, 1 Aug 2022 13:17:49 -0700 Subject: [PATCH 42/44] Resolve UDF failures --- tests/unit/test_context.py | 147 +++++++++++++++++++++++++++---------- 1 file changed, 109 insertions(+), 38 deletions(-) diff --git a/tests/unit/test_context.py b/tests/unit/test_context.py index f10025e5a..0b34146a4 100644 --- a/tests/unit/test_context.py +++ b/tests/unit/test_context.py @@ -4,8 +4,8 @@ import pandas as pd import pytest +from dask_planner.rust import SqlTypeName from dask_sql import Context -from dask_sql.mappings import python_to_sql_type from tests.utils import assert_eq try: @@ -196,11 +196,6 @@ def g(gpu=gpu): g(gpu=gpu) -int_sql_type = python_to_sql_type(int) -float_sql_type = python_to_sql_type(float) -str_sql_type = python_to_sql_type(str) - - def test_function_adding(): c = Context() @@ -214,12 +209,26 @@ def test_function_adding(): assert c.schema[c.schema_name].functions["f"].func == f assert len(c.schema[c.schema_name].function_lists) == 2 assert c.schema[c.schema_name].function_lists[0].name == "F" - assert c.schema[c.schema_name].function_lists[0].parameters == [("x", int_sql_type)] - assert c.schema[c.schema_name].function_lists[0].return_type == float_sql_type + assert c.schema[c.schema_name].function_lists[0].parameters[0][0] == "x" + assert ( + c.schema[c.schema_name].function_lists[0].parameters[0][1].getSqlType() + == SqlTypeName.BIGINT + ) + assert ( + c.schema[c.schema_name].function_lists[0].return_type.getSqlType() + == SqlTypeName.DOUBLE + ) assert not c.schema[c.schema_name].function_lists[0].aggregation assert c.schema[c.schema_name].function_lists[1].name == "f" - assert c.schema[c.schema_name].function_lists[1].parameters == [("x", int_sql_type)] - assert c.schema[c.schema_name].function_lists[1].return_type == float_sql_type + assert c.schema[c.schema_name].function_lists[1].parameters[0][0] == "x" + assert ( + c.schema[c.schema_name].function_lists[1].parameters[0][1].getSqlType() + == SqlTypeName.BIGINT + ) + assert ( + c.schema[c.schema_name].function_lists[1].return_type.getSqlType() + == SqlTypeName.DOUBLE + ) assert not c.schema[c.schema_name].function_lists[1].aggregation # Without replacement @@ -229,16 +238,26 @@ def test_function_adding(): assert c.schema[c.schema_name].functions["f"].func == f assert len(c.schema[c.schema_name].function_lists) == 4 assert c.schema[c.schema_name].function_lists[2].name == "F" - assert c.schema[c.schema_name].function_lists[2].parameters == [ - ("x", float_sql_type) - ] - assert c.schema[c.schema_name].function_lists[2].return_type == int_sql_type + assert c.schema[c.schema_name].function_lists[2].parameters[0][0] == "x" + assert ( + c.schema[c.schema_name].function_lists[2].parameters[0][1].getSqlType() + == SqlTypeName.DOUBLE + ) + assert ( + c.schema[c.schema_name].function_lists[2].return_type.getSqlType() + == SqlTypeName.BIGINT + ) assert not c.schema[c.schema_name].function_lists[2].aggregation assert c.schema[c.schema_name].function_lists[3].name == "f" - assert c.schema[c.schema_name].function_lists[3].parameters == [ - ("x", float_sql_type) - ] - assert c.schema[c.schema_name].function_lists[3].return_type == int_sql_type + assert c.schema[c.schema_name].function_lists[3].parameters[0][0] == "x" + assert ( + c.schema[c.schema_name].function_lists[3].parameters[0][1].getSqlType() + == SqlTypeName.DOUBLE + ) + assert ( + c.schema[c.schema_name].function_lists[3].return_type.getSqlType() + == SqlTypeName.BIGINT + ) assert not c.schema[c.schema_name].function_lists[3].aggregation # With replacement @@ -249,12 +268,26 @@ def test_function_adding(): assert c.schema[c.schema_name].functions["f"].func == f assert len(c.schema[c.schema_name].function_lists) == 2 assert c.schema[c.schema_name].function_lists[0].name == "F" - assert c.schema[c.schema_name].function_lists[0].parameters == [("x", str_sql_type)] - assert c.schema[c.schema_name].function_lists[0].return_type == str_sql_type + assert c.schema[c.schema_name].function_lists[0].parameters[0][0] == "x" + assert ( + c.schema[c.schema_name].function_lists[0].parameters[0][1].getSqlType() + == SqlTypeName.VARCHAR + ) + assert ( + c.schema[c.schema_name].function_lists[0].return_type.getSqlType() + == SqlTypeName.VARCHAR + ) assert not c.schema[c.schema_name].function_lists[0].aggregation assert c.schema[c.schema_name].function_lists[1].name == "f" - assert c.schema[c.schema_name].function_lists[1].parameters == [("x", str_sql_type)] - assert c.schema[c.schema_name].function_lists[1].return_type == str_sql_type + assert c.schema[c.schema_name].function_lists[1].parameters[0][0] == "x" + assert ( + c.schema[c.schema_name].function_lists[1].parameters[0][1].getSqlType() + == SqlTypeName.VARCHAR + ) + assert ( + c.schema[c.schema_name].function_lists[1].return_type.getSqlType() + == SqlTypeName.VARCHAR + ) assert not c.schema[c.schema_name].function_lists[1].aggregation @@ -271,12 +304,26 @@ def test_aggregation_adding(): assert c.schema[c.schema_name].functions["f"] == f assert len(c.schema[c.schema_name].function_lists) == 2 assert c.schema[c.schema_name].function_lists[0].name == "F" - assert c.schema[c.schema_name].function_lists[0].parameters == [("x", int_sql_type)] - assert c.schema[c.schema_name].function_lists[0].return_type == float_sql_type + assert c.schema[c.schema_name].function_lists[0].parameters[0][0] == "x" + assert ( + c.schema[c.schema_name].function_lists[0].parameters[0][1].getSqlType() + == SqlTypeName.BIGINT + ) + assert ( + c.schema[c.schema_name].function_lists[0].return_type.getSqlType() + == SqlTypeName.DOUBLE + ) assert c.schema[c.schema_name].function_lists[0].aggregation assert c.schema[c.schema_name].function_lists[1].name == "f" - assert c.schema[c.schema_name].function_lists[1].parameters == [("x", int_sql_type)] - assert c.schema[c.schema_name].function_lists[1].return_type == float_sql_type + assert c.schema[c.schema_name].function_lists[1].parameters[0][0] == "x" + assert ( + c.schema[c.schema_name].function_lists[1].parameters[0][1].getSqlType() + == SqlTypeName.BIGINT + ) + assert ( + c.schema[c.schema_name].function_lists[1].return_type.getSqlType() + == SqlTypeName.DOUBLE + ) assert c.schema[c.schema_name].function_lists[1].aggregation # Without replacement @@ -286,16 +333,26 @@ def test_aggregation_adding(): assert c.schema[c.schema_name].functions["f"] == f assert len(c.schema[c.schema_name].function_lists) == 4 assert c.schema[c.schema_name].function_lists[2].name == "F" - assert c.schema[c.schema_name].function_lists[2].parameters == [ - ("x", float_sql_type) - ] - assert c.schema[c.schema_name].function_lists[2].return_type == int_sql_type + assert c.schema[c.schema_name].function_lists[2].parameters[0][0] == "x" + assert ( + c.schema[c.schema_name].function_lists[2].parameters[0][1].getSqlType() + == SqlTypeName.DOUBLE + ) + assert ( + c.schema[c.schema_name].function_lists[2].return_type.getSqlType() + == SqlTypeName.BIGINT + ) assert c.schema[c.schema_name].function_lists[2].aggregation assert c.schema[c.schema_name].function_lists[3].name == "f" - assert c.schema[c.schema_name].function_lists[3].parameters == [ - ("x", float_sql_type) - ] - assert c.schema[c.schema_name].function_lists[3].return_type == int_sql_type + assert c.schema[c.schema_name].function_lists[3].parameters[0][0] == "x" + assert ( + c.schema[c.schema_name].function_lists[3].parameters[0][1].getSqlType() + == SqlTypeName.DOUBLE + ) + assert ( + c.schema[c.schema_name].function_lists[3].return_type.getSqlType() + == SqlTypeName.BIGINT + ) assert c.schema[c.schema_name].function_lists[3].aggregation # With replacement @@ -306,12 +363,26 @@ def test_aggregation_adding(): assert c.schema[c.schema_name].functions["f"] == f assert len(c.schema[c.schema_name].function_lists) == 2 assert c.schema[c.schema_name].function_lists[0].name == "F" - assert c.schema[c.schema_name].function_lists[0].parameters == [("x", str_sql_type)] - assert c.schema[c.schema_name].function_lists[0].return_type == str_sql_type + assert c.schema[c.schema_name].function_lists[0].parameters[0][0] == "x" + assert ( + c.schema[c.schema_name].function_lists[0].parameters[0][1].getSqlType() + == SqlTypeName.VARCHAR + ) + assert ( + c.schema[c.schema_name].function_lists[0].return_type.getSqlType() + == SqlTypeName.VARCHAR + ) assert c.schema[c.schema_name].function_lists[0].aggregation assert c.schema[c.schema_name].function_lists[1].name == "f" - assert c.schema[c.schema_name].function_lists[1].parameters == [("x", str_sql_type)] - assert c.schema[c.schema_name].function_lists[1].return_type == str_sql_type + assert c.schema[c.schema_name].function_lists[1].parameters[0][0] == "x" + assert ( + c.schema[c.schema_name].function_lists[1].parameters[0][1].getSqlType() + == SqlTypeName.VARCHAR + ) + assert ( + c.schema[c.schema_name].function_lists[1].return_type.getSqlType() + == SqlTypeName.VARCHAR + ) assert c.schema[c.schema_name].function_lists[1].aggregation From c55de4c290de34b7251e2048aceb7c6cc8409ec4 Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Mon, 1 Aug 2022 13:55:19 -0700 Subject: [PATCH 43/44] Resolve UDF failures for windows --- tests/unit/test_context.py | 55 +++++++++++++++++++++----------------- 1 file changed, 30 insertions(+), 25 deletions(-) diff --git a/tests/unit/test_context.py b/tests/unit/test_context.py index 0b34146a4..979ee0296 100644 --- a/tests/unit/test_context.py +++ b/tests/unit/test_context.py @@ -4,8 +4,8 @@ import pandas as pd import pytest -from dask_planner.rust import SqlTypeName from dask_sql import Context +from dask_sql.mappings import python_to_sql_type from tests.utils import assert_eq try: @@ -196,6 +196,11 @@ def g(gpu=gpu): g(gpu=gpu) +int_sql_type = python_to_sql_type(int).getSqlType() +float_sql_type = python_to_sql_type(float).getSqlType() +str_sql_type = python_to_sql_type(str).getSqlType() + + def test_function_adding(): c = Context() @@ -212,22 +217,22 @@ def test_function_adding(): assert c.schema[c.schema_name].function_lists[0].parameters[0][0] == "x" assert ( c.schema[c.schema_name].function_lists[0].parameters[0][1].getSqlType() - == SqlTypeName.BIGINT + == int_sql_type ) assert ( c.schema[c.schema_name].function_lists[0].return_type.getSqlType() - == SqlTypeName.DOUBLE + == float_sql_type ) assert not c.schema[c.schema_name].function_lists[0].aggregation assert c.schema[c.schema_name].function_lists[1].name == "f" assert c.schema[c.schema_name].function_lists[1].parameters[0][0] == "x" assert ( c.schema[c.schema_name].function_lists[1].parameters[0][1].getSqlType() - == SqlTypeName.BIGINT + == int_sql_type ) assert ( c.schema[c.schema_name].function_lists[1].return_type.getSqlType() - == SqlTypeName.DOUBLE + == float_sql_type ) assert not c.schema[c.schema_name].function_lists[1].aggregation @@ -241,22 +246,22 @@ def test_function_adding(): assert c.schema[c.schema_name].function_lists[2].parameters[0][0] == "x" assert ( c.schema[c.schema_name].function_lists[2].parameters[0][1].getSqlType() - == SqlTypeName.DOUBLE + == float_sql_type ) assert ( c.schema[c.schema_name].function_lists[2].return_type.getSqlType() - == SqlTypeName.BIGINT + == int_sql_type ) assert not c.schema[c.schema_name].function_lists[2].aggregation assert c.schema[c.schema_name].function_lists[3].name == "f" assert c.schema[c.schema_name].function_lists[3].parameters[0][0] == "x" assert ( c.schema[c.schema_name].function_lists[3].parameters[0][1].getSqlType() - == SqlTypeName.DOUBLE + == float_sql_type ) assert ( c.schema[c.schema_name].function_lists[3].return_type.getSqlType() - == SqlTypeName.BIGINT + == int_sql_type ) assert not c.schema[c.schema_name].function_lists[3].aggregation @@ -271,22 +276,22 @@ def test_function_adding(): assert c.schema[c.schema_name].function_lists[0].parameters[0][0] == "x" assert ( c.schema[c.schema_name].function_lists[0].parameters[0][1].getSqlType() - == SqlTypeName.VARCHAR + == str_sql_type ) assert ( c.schema[c.schema_name].function_lists[0].return_type.getSqlType() - == SqlTypeName.VARCHAR + == str_sql_type ) assert not c.schema[c.schema_name].function_lists[0].aggregation assert c.schema[c.schema_name].function_lists[1].name == "f" assert c.schema[c.schema_name].function_lists[1].parameters[0][0] == "x" assert ( c.schema[c.schema_name].function_lists[1].parameters[0][1].getSqlType() - == SqlTypeName.VARCHAR + == str_sql_type ) assert ( c.schema[c.schema_name].function_lists[1].return_type.getSqlType() - == SqlTypeName.VARCHAR + == str_sql_type ) assert not c.schema[c.schema_name].function_lists[1].aggregation @@ -307,22 +312,22 @@ def test_aggregation_adding(): assert c.schema[c.schema_name].function_lists[0].parameters[0][0] == "x" assert ( c.schema[c.schema_name].function_lists[0].parameters[0][1].getSqlType() - == SqlTypeName.BIGINT + == int_sql_type ) assert ( c.schema[c.schema_name].function_lists[0].return_type.getSqlType() - == SqlTypeName.DOUBLE + == float_sql_type ) assert c.schema[c.schema_name].function_lists[0].aggregation assert c.schema[c.schema_name].function_lists[1].name == "f" assert c.schema[c.schema_name].function_lists[1].parameters[0][0] == "x" assert ( c.schema[c.schema_name].function_lists[1].parameters[0][1].getSqlType() - == SqlTypeName.BIGINT + == int_sql_type ) assert ( c.schema[c.schema_name].function_lists[1].return_type.getSqlType() - == SqlTypeName.DOUBLE + == float_sql_type ) assert c.schema[c.schema_name].function_lists[1].aggregation @@ -336,22 +341,22 @@ def test_aggregation_adding(): assert c.schema[c.schema_name].function_lists[2].parameters[0][0] == "x" assert ( c.schema[c.schema_name].function_lists[2].parameters[0][1].getSqlType() - == SqlTypeName.DOUBLE + == float_sql_type ) assert ( c.schema[c.schema_name].function_lists[2].return_type.getSqlType() - == SqlTypeName.BIGINT + == int_sql_type ) assert c.schema[c.schema_name].function_lists[2].aggregation assert c.schema[c.schema_name].function_lists[3].name == "f" assert c.schema[c.schema_name].function_lists[3].parameters[0][0] == "x" assert ( c.schema[c.schema_name].function_lists[3].parameters[0][1].getSqlType() - == SqlTypeName.DOUBLE + == float_sql_type ) assert ( c.schema[c.schema_name].function_lists[3].return_type.getSqlType() - == SqlTypeName.BIGINT + == int_sql_type ) assert c.schema[c.schema_name].function_lists[3].aggregation @@ -366,22 +371,22 @@ def test_aggregation_adding(): assert c.schema[c.schema_name].function_lists[0].parameters[0][0] == "x" assert ( c.schema[c.schema_name].function_lists[0].parameters[0][1].getSqlType() - == SqlTypeName.VARCHAR + == str_sql_type ) assert ( c.schema[c.schema_name].function_lists[0].return_type.getSqlType() - == SqlTypeName.VARCHAR + == str_sql_type ) assert c.schema[c.schema_name].function_lists[0].aggregation assert c.schema[c.schema_name].function_lists[1].name == "f" assert c.schema[c.schema_name].function_lists[1].parameters[0][0] == "x" assert ( c.schema[c.schema_name].function_lists[1].parameters[0][1].getSqlType() - == SqlTypeName.VARCHAR + == str_sql_type ) assert ( c.schema[c.schema_name].function_lists[1].return_type.getSqlType() - == SqlTypeName.VARCHAR + == str_sql_type ) assert c.schema[c.schema_name].function_lists[1].aggregation From 75a24495da87f74c6a171e362acf3fa339ad22eb Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Mon, 1 Aug 2022 13:55:43 -0700 Subject: [PATCH 44/44] Remove calcite-specific reinterpret --- dask_sql/physical/rex/core/call.py | 1 - 1 file changed, 1 deletion(-) diff --git a/dask_sql/physical/rex/core/call.py b/dask_sql/physical/rex/core/call.py index 77430b562..9ee59ba0d 100644 --- a/dask_sql/physical/rex/core/call.py +++ b/dask_sql/physical/rex/core/call.py @@ -876,7 +876,6 @@ class RexCallPlugin(BaseRexPlugin): "/int": IntDivisionOperator(), # special operations "cast": CastOperation(), - "reinterpret": CastOperation(), "case": CaseOperation(), "not like": NotOperation().of(LikeOperation()), "like": LikeOperation(),