diff --git a/ci/deps/dask-min.yml b/ci/deps/dask-min.yml new file mode 100644 index 000000000000..edaa64e3f415 --- /dev/null +++ b/ci/deps/dask-min.yml @@ -0,0 +1 @@ + dask=2.22.0 diff --git a/ci/deps/dask.yml b/ci/deps/dask.yml index b2034ba3b6cd..832e8ddfb02d 100644 --- a/ci/deps/dask.yml +++ b/ci/deps/dask.yml @@ -1 +1 @@ -dask + dask diff --git a/environment.yml b/environment.yml index ce7960440e22..8ba5c78097fa 100644 --- a/environment.yml +++ b/environment.yml @@ -14,7 +14,7 @@ dependencies: # Ibis soft dependencies # TODO This section is probably not very accurate right now (some dependencies should probably be in the backends files) - sqlalchemy>=1.3 - - dask + - dask>=2.22.0 - graphviz>=2.38 - openjdk=8 - pytables>=3.6 diff --git a/ibis/__init__.py b/ibis/__init__.py index d315b90acf6d..d2c4b998433b 100644 --- a/ibis/__init__.py +++ b/ibis/__init__.py @@ -83,7 +83,7 @@ with suppress(ImportError): # pip install ibis-framework[dask] - import ibis.backends as dask # noqa: F401 + from ibis.backends import dask # noqa: F401 __version__ = get_versions()['version'] diff --git a/ibis/backends/dask/client.py b/ibis/backends/dask/client.py index a3135b9cd8a4..8db5038da0f5 100644 --- a/ibis/backends/dask/client.py +++ b/ibis/backends/dask/client.py @@ -6,12 +6,12 @@ from functools import partial from typing import Dict, List, Mapping +import dask import dask.dataframe as dd import dateutil.parser import numpy as np import pandas as pd import toolz -from multipledispatch import Dispatcher from pandas.api.types import DatetimeTZDtype from pkg_resources import parse_version @@ -25,11 +25,13 @@ PANDAS_DATE_TYPES, PANDAS_STRING_TYPES, _inferable_pandas_dtypes, + convert, convert_timezone, ibis_dtype_to_pandas, ibis_schema_to_pandas, ) -from ibis.backends.pandas.core import execute_and_reset + +from .core import execute_and_reset infer_dask_dtype = pd.api.types.infer_dtype @@ -82,35 +84,13 @@ def infer_dask_schema(df, schema=None): ibis_schema_to_dask = ibis_schema_to_pandas -convert = Dispatcher( - 'convert', - doc="""\ -Convert `column` to the dask dtype corresponding to `out_dtype`, where the -dtype of `column` is `in_dtype`. - -Parameters ----------- -in_dtype : Union[np.dtype, dask_dtype] - The dtype of `column`, used for dispatching -out_dtype : ibis.expr.datatypes.DataType - The requested ibis type of the output -column : dd.Series - The column to convert - -Returns -------- -result : dd.Series - The converted column -""", -) - @convert.register(DatetimeTZDtype, dt.Timestamp, dd.Series) def convert_datetimetz_to_timestamp(in_dtype, out_dtype, column): output_timezone = out_dtype.timezone if output_timezone is not None: return column.dt.tz_convert(output_timezone) - return column.astype(out_dtype.to_dask(), errors='ignore') + return column.astype(out_dtype.to_dask()) DASK_STRING_TYPES = PANDAS_STRING_TYPES @@ -151,7 +131,7 @@ def convert_any_to_interval(_, out_dtype, column): @convert.register(np.dtype, dt.String, dd.Series) def convert_any_to_string(_, out_dtype, column): - result = column.astype(out_dtype.to_dask(), errors='ignore') + result = column.astype(out_dtype.to_dask()) return result @@ -167,46 +147,11 @@ def convert_boolean_to_series(in_dtype, out_dtype, column): @convert.register(object, dt.DataType, dd.Series) def convert_any_to_any(_, out_dtype, column): - return column.astype(out_dtype.to_dask(), errors='ignore') - - -def ibis_schema_apply_to(schema: sch.Schema, df: dd.DataFrame) -> dd.DataFrame: - """Applies the Ibis schema to a dask DataFrame - - Parameters - ---------- - schema : ibis.schema.Schema - df : dask.dataframe.DataFrame - - Returns - ------- - df : dask.dataframeDataFrame - - Notes - ----- - Mutates `df` - """ - - for column, dtype in schema.items(): - dask_dtype = dtype.to_dask() - col = df[column] - col_dtype = col.dtype - - try: - not_equal = dask_dtype != col_dtype - except TypeError: - # ugh, we can't compare dtypes coming from dask, assume not equal - not_equal = True - - if not_equal or isinstance(dtype, dt.String): - df[column] = convert(col_dtype, dtype, col) - - return df + return column.astype(out_dtype.to_dask()) dt.DataType.to_dask = ibis_dtype_to_dask sch.Schema.to_dask = ibis_schema_to_dask -sch.Schema.apply_to = ibis_schema_apply_to class DaskTable(ops.DatabaseTable): @@ -297,7 +242,6 @@ def create_table( if obj is not None: df = obj else: - # TODO - this isn't right dtypes = ibis_schema_to_dask(schema) df = schema.apply_to( dd.from_pandas( @@ -342,4 +286,4 @@ def exists_table(self, name: str) -> bool: @property def version(self) -> str: """Return the version of the underlying backend library.""" - return parse_version(dd.__version__) + return parse_version(dask.__version__) diff --git a/ibis/backends/dask/core.py b/ibis/backends/dask/core.py index 0371ff632f91..04deb426c579 100644 --- a/ibis/backends/dask/core.py +++ b/ibis/backends/dask/core.py @@ -110,11 +110,76 @@ from __future__ import absolute_import +from typing import Optional + import dask.dataframe as dd from ibis.backends.pandas.core import ( + execute, is_computable_input, is_computable_input_arg, ) +from ibis.expr.typing import TimeContext is_computable_input.register(dd.core.Scalar)(is_computable_input_arg) + + +def execute_and_reset( + expr, + params=None, + scope=None, + timecontext: Optional[TimeContext] = None, + aggcontext=None, + **kwargs, +): + """Execute an expression against data that are bound to it. If no data + are bound, raise an Exception. + Notes + ----- + The difference between this function and :func:`~ibis.dask.core.execute` + is that this function resets the index of the result, if the result has + an index. + Parameters + ---------- + expr : ibis.expr.types.Expr + The expression to execute + params : Mapping[ibis.expr.types.Expr, object] + The data that an unbound parameter in `expr` maps to + scope : Mapping[ibis.expr.operations.Node, object] + Additional scope, mapping ibis operations to data + timecontext : Optional[TimeContext] + timecontext needed for execution + aggcontext : Optional[ibis.dask.aggcontext.AggregationContext] + An object indicating how to compute aggregations. For example, + a rolling mean needs to be computed differently than the mean of a + column. + kwargs : Dict[str, object] + Additional arguments that can potentially be used by individual node + execution + Returns + ------- + result : Union[ + dask.dataframe.Series, + dask.dataframe.DataFrame, + ibis.dask.core.simple_types + ] + Raises + ------ + ValueError + * If no data are bound to the input expression + """ + result = execute( + expr, + params=params, + scope=scope, + timecontext=timecontext, + aggcontext=aggcontext, + **kwargs, + ) + if isinstance(result, dd.DataFrame): + schema = expr.schema() + df = result.reset_index() + return df[schema.names] + elif isinstance(result, dd.Series): + return result.reset_index(drop=True) + return result diff --git a/ibis/backends/dask/execution/__init__.py b/ibis/backends/dask/execution/__init__.py index 431f99710d45..2b86fd8a143b 100644 --- a/ibis/backends/dask/execution/__init__.py +++ b/ibis/backends/dask/execution/__init__.py @@ -1,3 +1,13 @@ -# This is awaiting implementation of the rest of execution -# https://github.com/ibis-project/ibis/issues/2537 -from ibis.backends.pandas.core import execute, execute_node # noqa: F401,F403 +from .aggregations import * # noqa: F401,F403 +from .arrays import * # noqa: F401,F403 +from .decimal import * # noqa: F401,F403 +from .generic import * # noqa: F401,F403 +from .indexing import * # noqa: F401,F403 +from .join import * # noqa: F401,F403 +from .maps import * # noqa: F401,F403 +from .numeric import * # noqa: F401,F403 +from .reductions import * # noqa: F401,F403 +from .selection import * # noqa: F401,F403 +from .strings import * # noqa: F401,F403 +from .structs import * # noqa: F401,F403 +from .temporal import * # noqa: F401,F403 diff --git a/ibis/backends/dask/execution/aggregations.py b/ibis/backends/dask/execution/aggregations.py new file mode 100644 index 000000000000..53dfada78a07 --- /dev/null +++ b/ibis/backends/dask/execution/aggregations.py @@ -0,0 +1,143 @@ +"""Execution rules for Aggregatons - mostly TODO + +- ops.Aggregation +- ops.Any +- ops.NotAny +- ops.All +- ops.NotAll + +""" + +import functools +import operator +from typing import Optional + +import dask.dataframe as dd + +import ibis.expr.operations as ops +from ibis.backends.pandas.execution.generic import execute, execute_node +from ibis.expr.scope import Scope +from ibis.expr.typing import TimeContext + + +# TODO - aggregations - #2553 +# Not all code paths work cleanly here +@execute_node.register(ops.Aggregation, dd.DataFrame) +def execute_aggregation_dataframe( + op, data, scope=None, timecontext: Optional[TimeContext] = None, **kwargs +): + assert op.metrics, 'no metrics found during aggregation execution' + + if op.sort_keys: + raise NotImplementedError( + 'sorting on aggregations not yet implemented' + ) + + predicates = op.predicates + if predicates: + predicate = functools.reduce( + operator.and_, + ( + execute(p, scope=scope, timecontext=timecontext, **kwargs) + for p in predicates + ), + ) + data = data.loc[predicate] + + columns = {} + + if op.by: + grouping_key_pairs = list( + zip(op.by, map(operator.methodcaller('op'), op.by)) + ) + grouping_keys = [ + by_op.name + if isinstance(by_op, ops.TableColumn) + else execute( + by, scope=scope, timecontext=timecontext, **kwargs + ).rename(by.get_name()) + for by, by_op in grouping_key_pairs + ] + columns.update( + (by_op.name, by.get_name()) + for by, by_op in grouping_key_pairs + if hasattr(by_op, 'name') + ) + source = data.groupby(grouping_keys) + else: + source = data + + scope = scope.merge_scope(Scope({op.table.op(): source}, timecontext)) + + pieces = [] + for metric in op.metrics: + piece = execute(metric, scope=scope, timecontext=timecontext, **kwargs) + piece.name = metric.get_name() + pieces.append(piece) + + result = dd.concat(pieces, axis=1) + + # If grouping, need a reset to get the grouping key back as a column + if op.by: + result = result.reset_index() + + result.columns = [columns.get(c, c) for c in result.columns] + + if op.having: + # .having(...) is only accessible on groupby, so this should never + # raise + if not op.by: + raise ValueError( + 'Filtering out aggregation values is not allowed without at ' + 'least one grouping key' + ) + + # TODO(phillipc): Don't recompute identical subexpressions + predicate = functools.reduce( + operator.and_, + ( + execute(having, scope=scope, timecontext=timecontext, **kwargs) + for having in op.having + ), + ) + assert len(predicate) == len( + result + ), 'length of predicate does not match length of DataFrame' + result = result.loc[predicate.values] + return result + + +# TODO - aggregations - #2553 +# @execute_node.register((ops.Any, ops.All), (dd.Series, SeriesGroupBy)) +# def execute_any_all_series(op, data, aggcontext=None, **kwargs): +# if isinstance(aggcontext, (agg_ctx.Summarize, agg_ctx.Transform)): +# result = aggcontext.agg(data, type(op).__name__.lower()) +# else: +# result = aggcontext.agg( +# data, lambda data: getattr(data, type(op).__name__.lower())() +# ) +# return result + +# TODO - aggregations - #2553 +# @execute_node.register(ops.NotAny, (dd.Series, SeriesGroupBy)) +# def execute_notany_series(op, data, aggcontext=None, **kwargs): +# if isinstance(aggcontext, (agg_ctx.Summarize, agg_ctx.Transform)): +# result = ~(aggcontext.agg(data, 'any')) +# else: +# result = aggcontext.agg(data, lambda data: ~(data.any())) +# try: +# return result.astype(bool) +# except TypeError: +# return result + +# TODO - aggregations - #2553 +# @execute_node.register(ops.NotAll, (dd.Series, SeriesGroupBy)) +# def execute_notall_series(op, data, aggcontext=None, **kwargs): +# if isinstance(aggcontext, (agg_ctx.Summarize, agg_ctx.Transform)): +# result = ~(aggcontext.agg(data, 'all')) +# else: +# result = aggcontext.agg(data, lambda data: ~(data.all())) +# try: +# return result.astype(bool) +# except TypeError: +# return result diff --git a/ibis/backends/dask/execution/arrays.py b/ibis/backends/dask/execution/arrays.py new file mode 100644 index 000000000000..0e47759d3002 --- /dev/null +++ b/ibis/backends/dask/execution/arrays.py @@ -0,0 +1,56 @@ +import itertools + +import dask.dataframe as dd +import dask.dataframe.groupby as ddgb + +import ibis.expr.operations as ops +from ibis.backends.pandas.execution.arrays import ( + execute_array_concat, + execute_array_index, + execute_array_length, + execute_array_repeat, + execute_array_slice, + execute_node, +) + +from .util import TypeRegistrationDict, register_types_to_dispatcher + +DASK_DISPATCH_TYPES: TypeRegistrationDict = { + ops.ArrayLength: [((dd.Series,), execute_array_length)], + ops.ArrayConcat: [ + ((dd.Series, (dd.Series, list)), execute_array_concat), + ((list, dd.Series), execute_array_concat), + ], + ops.ArrayIndex: [((dd.Series, int), execute_array_index)], + ops.ArrayRepeat: [ + ((dd.Series, dd.Series), execute_array_repeat), + ((int, (dd.Series, list)), execute_array_repeat), + (((dd.Series, list), int), execute_array_repeat), + ], + ops.ArraySlice: [ + ((dd.Series, int, (int, type(None))), execute_array_slice), + ], +} + +register_types_to_dispatcher(execute_node, DASK_DISPATCH_TYPES) + + +collect_list = dd.Aggregation( + name="collect_list", + chunk=lambda s: s.apply(list), + agg=lambda s0: s0.apply( + lambda chunks: list(itertools.chain.from_iterable(chunks)) + ), +) + + +# TODO - aggregations - #2553 +@execute_node.register(ops.ArrayCollect, dd.Series) +def execute_array_collect(op, data, aggcontext=None, **kwargs): + return aggcontext.agg(data, collect_list) + + +# TODO - grouping - #2553 +@execute_node.register(ops.ArrayCollect, ddgb.SeriesGroupBy) +def execute_array_collect_grouped_series(op, data, aggcontext=None, **kwargs): + return data.agg(collect_list) diff --git a/ibis/backends/dask/execution/decimal.py b/ibis/backends/dask/execution/decimal.py new file mode 100644 index 000000000000..a23272c59ef3 --- /dev/null +++ b/ibis/backends/dask/execution/decimal.py @@ -0,0 +1,25 @@ +from __future__ import absolute_import + +import decimal + +import dask.dataframe as dd + +import ibis.expr.datatypes as dt +import ibis.expr.operations as ops +from ibis.backends.pandas.execution.decimal import execute_node + + +@execute_node.register(ops.Cast, dd.Series, dt.Decimal) +def execute_cast_series_to_decimal(op, data, type, **kwargs): + precision = type.precision + scale = type.scale + context = decimal.Context(prec=precision) + places = context.create_decimal( + '{}.{}'.format('0' * (precision - scale), '0' * scale) + ) + return data.apply( + lambda x, context=context, places=places: ( # noqa: E501 + context.create_decimal(x).quantize(places) + ), + meta=(data.name, "object"), + ) diff --git a/ibis/backends/dask/execution/generic.py b/ibis/backends/dask/execution/generic.py new file mode 100644 index 000000000000..ba77450cc0af --- /dev/null +++ b/ibis/backends/dask/execution/generic.py @@ -0,0 +1,391 @@ +"""Execution rules for generic ibis operations.""" + +import collections +import datetime +import decimal +import numbers + +import dask.array as da +import dask.dataframe as dd +import dask.dataframe.groupby as ddgb +import numpy as np +import pandas as pd +from pandas import isnull, to_datetime +from pandas.api.types import DatetimeTZDtype + +import ibis.common.exceptions as com +import ibis.expr.datatypes as dt +import ibis.expr.operations as ops +import ibis.expr.types as ir +from ibis.backends.pandas.core import ( + integer_types, + numeric_types, + simple_types, +) +from ibis.backends.pandas.execution import constants +from ibis.backends.pandas.execution.generic import ( + execute_between, + execute_cast_series_array, + execute_cast_series_generic, + execute_count_frame, + execute_count_frame_groupby, + execute_database_table_client, + execute_difference_dataframe_dataframe, + execute_distinct_dataframe, + execute_intersection_dataframe_dataframe, + execute_isinf, + execute_isnan, + execute_node, + execute_node_contains_series_sequence, + execute_node_ifnull_series, + execute_node_not_contains_series_sequence, + execute_node_nullif_series, + execute_node_nullif_series_scalar, + execute_node_self_reference_dataframe, + execute_null_if_zero_series, + execute_series_clip, + execute_series_isnull, + execute_series_notnnull, + execute_sort_key_series_bool, + execute_string_split, + execute_table_column_df_or_df_groupby, +) + +from ..client import DaskClient, DaskTable +from .util import TypeRegistrationDict, register_types_to_dispatcher + +# Many dask and pandas functions are functionally equivalent, so we just add +# on registrations for dask types +DASK_DISPATCH_TYPES: TypeRegistrationDict = { + ops.Cast: [ + ((dd.Series, dt.DataType), execute_cast_series_generic), + ((dd.Series, dt.Array), execute_cast_series_array), + ], + ops.SortKey: [((dd.Series, bool), execute_sort_key_series_bool)], + ops.Clip: [ + ( + ( + dd.Series, + (dd.Series, type(None)) + numeric_types, + (dd.Series, type(None)) + numeric_types, + ), + execute_series_clip, + ), + ], + ops.TableColumn: [ + ( + ((dd.DataFrame, ddgb.DataFrameGroupBy),), + execute_table_column_df_or_df_groupby, + ), + ], + ops.Count: [ + ((ddgb.DataFrameGroupBy, type(None)), execute_count_frame_groupby), + ((dd.DataFrame, type(None)), execute_count_frame), + ], + ops.NullIfZero: [((dd.Series,), execute_null_if_zero_series)], + ops.StringSplit: [((dd.Series, (dd.Series, str)), execute_string_split)], + ops.Between: [ + ( + ( + dd.Series, + (dd.Series, numbers.Real, str, datetime.datetime), + (dd.Series, numbers.Real, str, datetime.datetime), + ), + execute_between, + ), + ], + ops.Intersection: [ + ( + (dd.DataFrame, dd.DataFrame), + execute_intersection_dataframe_dataframe, + ) + ], + ops.Difference: [ + ((dd.DataFrame, dd.DataFrame), execute_difference_dataframe_dataframe) + ], + ops.IsNull: [((dd.Series,), execute_series_isnull)], + ops.NotNull: [((dd.Series,), execute_series_notnnull)], + ops.IsNan: [((dd.Series,), execute_isnan)], + ops.IsInf: [((dd.Series,), execute_isinf)], + ops.SelfReference: [ + ((dd.DataFrame,), execute_node_self_reference_dataframe) + ], + ops.Contains: [ + ( + (dd.Series, (collections.abc.Sequence, collections.abc.Set)), + execute_node_contains_series_sequence, + ) + ], + ops.NotContains: [ + ( + (dd.Series, (collections.abc.Sequence, collections.abc.Set)), + execute_node_not_contains_series_sequence, + ) + ], + ops.IfNull: [ + ((dd.Series, simple_types), execute_node_ifnull_series), + ((dd.Series, dd.Series), execute_node_ifnull_series), + ], + ops.NullIf: [ + ((dd.Series, dd.Series), execute_node_nullif_series), + ((dd.Series, simple_types), execute_node_nullif_series_scalar), + ], + ops.Distinct: [((dd.DataFrame,), execute_distinct_dataframe)], +} +register_types_to_dispatcher(execute_node, DASK_DISPATCH_TYPES) + +execute_node.register(DaskTable, DaskClient)(execute_database_table_client) + + +@execute_node.register(ops.Arbitrary, dd.Series, (dd.Series, type(None))) +def execute_arbitrary_series_mask(op, data, mask, aggcontext=None, **kwargs): + """ + Note: we cannot use the pandas version because Dask does not support .iloc + """ + if op.how == 'first': + index = 0 + elif op.how == 'last': + index = -1 + else: + raise com.OperationNotDefinedError( + 'Arbitrary {!r} is not supported'.format(op.how) + ) + + data = data[mask] if mask is not None else data + return data.loc[index] + + +# TODO - grouping - #2553 +@execute_node.register(ops.Cast, ddgb.SeriesGroupBy, dt.DataType) +def execute_cast_series_group_by(op, data, type, **kwargs): + result = execute_cast_series_generic(op, data.obj, type, **kwargs) + return result.groupby(data.grouper.groupings) + + +@execute_node.register(ops.Cast, dd.Series, dt.Timestamp) +def execute_cast_series_timestamp(op, data, type, **kwargs): + arg = op.arg + from_type = arg.type() + + if from_type.equals(type): # noop cast + return data + + tz = type.timezone + + if isinstance(from_type, (dt.Timestamp, dt.Date)): + return data.astype( + 'M8[ns]' if tz is None else DatetimeTZDtype('ns', tz) + ) + + if isinstance(from_type, (dt.String, dt.Integer)): + timestamps = data.map_partitions( + to_datetime, + infer_datetime_format=True, + meta=(data.name, 'datetime64[ns]'), + ) + # TODO - is there a better way to do this + timestamps = timestamps.astype(timestamps.head(1).dtype) + if getattr(timestamps.dtype, "tz", None) is not None: + return timestamps.dt.tz_convert(tz) + else: + return timestamps.dt.tz_localize(tz) + + raise TypeError("Don't know how to cast {} to {}".format(from_type, type)) + + +@execute_node.register(ops.Cast, dd.Series, dt.Date) +def execute_cast_series_date(op, data, type, **kwargs): + arg = op.args[0] + from_type = arg.type() + + if from_type.equals(type): + return data + + # TODO - we return slightly different things depending on the branch + # double check what the logic should be + + if isinstance(from_type, dt.Timestamp): + return data.dt.normalize() + + if from_type.equals(dt.string): + # TODO - this is broken + datetimes = data.map_partitions( + to_datetime, + infer_datetime_format=True, + meta=(data.name, 'datetime64[ns]'), + ) + + # TODO - we are getting rid of the index here + return datetimes.dt.normalize() + + if isinstance(from_type, dt.Integer): + return data.map_partitions( + to_datetime, unit='D', meta=(data.name, 'datetime64[ns]') + ) + + raise TypeError("Don't know how to cast {} to {}".format(from_type, type)) + + +@execute_node.register(ops.Limit, dd.DataFrame, integer_types, integer_types) +def execute_limit_frame(op, data, nrows, offset, **kwargs): + # NOTE: Dask Dataframes do not support iloc row based indexing + return data.loc[offset : offset + nrows] + + +@execute_node.register(ops.Not, (dd.core.Scalar, dd.Series)) +def execute_not_scalar_or_series(op, data, **kwargs): + return ~data + + +@execute_node.register(ops.BinaryOp, dd.Series, dd.Series) +@execute_node.register(ops.BinaryOp, dd.Series, dd.core.Scalar) +@execute_node.register(ops.BinaryOp, dd.core.Scalar, dd.Series) +@execute_node.register( + (ops.NumericBinaryOp, ops.LogicalBinaryOp, ops.Comparison), + numeric_types, + dd.Series, +) +@execute_node.register( + (ops.NumericBinaryOp, ops.LogicalBinaryOp, ops.Comparison), + dd.Series, + numeric_types, +) +@execute_node.register((ops.Comparison, ops.Add, ops.Multiply), dd.Series, str) +@execute_node.register((ops.Comparison, ops.Add, ops.Multiply), str, dd.Series) +def execute_binary_op(op, left, right, **kwargs): + op_type = type(op) + try: + operation = constants.BINARY_OPERATIONS[op_type] + except KeyError: + raise NotImplementedError( + 'Binary operation {} not implemented'.format(op_type.__name__) + ) + else: + return operation(left, right) + + +# TODO - grouping - #2553 +@execute_node.register(ops.BinaryOp, ddgb.SeriesGroupBy, ddgb.SeriesGroupBy) +def execute_binary_op_series_group_by(op, left, right, **kwargs): + left_groupings = left.grouper.groupings + right_groupings = right.grouper.groupings + if left_groupings != right_groupings: + raise ValueError( + 'Cannot perform {} operation on two series with ' + 'different groupings'.format(type(op).__name__) + ) + result = execute_binary_op(op, left.obj, right.obj, **kwargs) + return result.groupby(left_groupings) + + +@execute_node.register(ops.BinaryOp, ddgb.SeriesGroupBy, simple_types) +def execute_binary_op_series_gb_simple(op, left, right, **kwargs): + op_type = type(op) + try: + operation = constants.BINARY_OPERATIONS[op_type] + except KeyError: + raise NotImplementedError( + 'Binary operation {} not implemented'.format(op_type.__name__) + ) + else: + return left.apply(lambda x, op=operation, right=right: op(x, right)) + + +# TODO - grouping - #2553 +@execute_node.register(ops.BinaryOp, simple_types, ddgb.SeriesGroupBy) +def execute_binary_op_simple_series_gb(op, left, right, **kwargs): + result = execute_binary_op(op, left, right, **kwargs) + return result.groupby(right.grouper.groupings) + + +# TODO - grouping - #2553 +@execute_node.register(ops.UnaryOp, ddgb.SeriesGroupBy) +def execute_unary_op_series_gb(op, operand, **kwargs): + result = execute_node(op, operand.obj, **kwargs) + return result + + +# TODO - grouping - #2553 +@execute_node.register( + (ops.Log, ops.Round), + ddgb.SeriesGroupBy, + (numbers.Real, decimal.Decimal, type(None)), +) +def execute_log_series_gb_others(op, left, right, **kwargs): + result = execute_node(op, left.obj, right, **kwargs) + return result.groupby(left.grouper.groupings) + + +# TODO - grouping - #2553 +@execute_node.register( + (ops.Log, ops.Round), ddgb.SeriesGroupBy, ddgb.SeriesGroupBy +) +def execute_log_series_gb_series_gb(op, left, right, **kwargs): + result = execute_node(op, left.obj, right.obj, **kwargs) + return result.groupby(left.grouper.groupings) + + +@execute_node.register(ops.DistinctColumn, dd.Series) +def execute_series_distinct(op, data, **kwargs): + return data.unique() + + +@execute_node.register(ops.Union, dd.DataFrame, dd.DataFrame, bool) +def execute_union_dataframe_dataframe( + op, left: dd.DataFrame, right: dd.DataFrame, distinct, **kwargs +): + result = dd.concat([left, right], axis=0) + return result.drop_duplicates() if distinct else result + + +@execute_node.register(ops.IfNull, simple_types, dd.Series) +def execute_node_ifnull_scalar_series(op, value, replacement, **kwargs): + return ( + replacement + if isnull(value) + else dd.from_pandas( + pd.Series(value, index=replacement.index), + npartitions=replacement.npartitions, + ) + ) + + +@execute_node.register(ops.NullIf, simple_types, dd.Series) +def execute_node_nullif_scalar_series(op, value, series, **kwargs): + # TODO - not preserving the index + return dd.from_array(da.where(series.eq(value).values, np.nan, value)) + + +def wrap_case_result(raw: np.ndarray, expr: ir.ValueExpr): + """Wrap a CASE statement result in a Series and handle returning scalars. + + Parameters + ---------- + raw : ndarray[T] + The raw results of executing the ``CASE`` expression + expr : ValueExpr + The expression from the which `raw` was computed + + Returns + ------- + Union[scalar, Series] + """ + raw_1d = np.atleast_1d(raw) + if np.any(isnull(raw_1d)): + result = dd.from_array(raw_1d) + else: + result = dd.from_array( + raw_1d.astype(constants.IBIS_TYPE_TO_PANDAS_TYPE[expr.type()]) + ) + # TODO - we force computation here + if isinstance(expr, ir.ScalarExpr) and result.size.compute() == 1: + return result.head().item() + return result + + +@execute_node.register(ops.SimpleCase, dd.Series, list, list, object) +def execute_simple_case_series(op, value, whens, thens, otherwise, **kwargs): + if otherwise is None: + otherwise = np.nan + raw = np.select([value == when for when in whens], thens, otherwise) + return wrap_case_result(raw, op.to_expr()) diff --git a/ibis/backends/dask/execution/indexing.py b/ibis/backends/dask/execution/indexing.py new file mode 100644 index 000000000000..b6dc996d2aed --- /dev/null +++ b/ibis/backends/dask/execution/indexing.py @@ -0,0 +1,58 @@ +"""Execution rules for ops.Where operations""" + +import dask.dataframe as dd +import numpy as np + +import ibis.expr.operations as ops +from ibis.backends.pandas.core import boolean_types, scalar_types +from ibis.backends.pandas.execution.generic import ( + execute_node, + execute_node_where_scalar_scalar_scalar, + execute_node_where_series_series_series, +) + +from .util import TypeRegistrationDict, register_types_to_dispatcher + +DASK_DISPATCH_TYPES: TypeRegistrationDict = { + ops.Where: [ + ( + (dd.Series, dd.Series, dd.Series), + execute_node_where_series_series_series, + ), + ( + (dd.Series, dd.Series, scalar_types), + execute_node_where_series_series_series, + ), + ( + (boolean_types, dd.Series, dd.Series,), + execute_node_where_scalar_scalar_scalar, + ), + ] +} +register_types_to_dispatcher(execute_node, DASK_DISPATCH_TYPES) + + +def execute_node_where_series_scalar_scalar(op, cond, true, false, **kwargs): + return dd.from_array(np.repeat(true, len(cond))).where(cond, other=false) + + +for scalar_type in scalar_types: + execute_node.register(ops.Where, dd.Series, scalar_type, scalar_type)( + execute_node_where_series_scalar_scalar + ) + + +@execute_node.register(ops.Where, boolean_types, dd.Series, scalar_types) +def execute_node_where_scalar_series_scalar(op, cond, true, false, **kwargs): + if cond: + return true + else: + # TODO double check this is the right way to do this + out = dd.from_array(np.repeat(false, len(true))) + out.index = true.index + return out + + +@execute_node.register(ops.Where, boolean_types, scalar_types, dd.Series) +def execute_node_where_scalar_scalar_series(op, cond, true, false, **kwargs): + return dd.from_array(np.repeat(true, len(false))) if cond else false diff --git a/ibis/backends/dask/execution/join.py b/ibis/backends/dask/execution/join.py new file mode 100644 index 000000000000..9d03dc242c18 --- /dev/null +++ b/ibis/backends/dask/execution/join.py @@ -0,0 +1,109 @@ +import operator + +import dask.dataframe as dd +from pandas import Timedelta + +import ibis.expr.operations as ops +import ibis.util +from ibis.backends.pandas.execution.join import ( + _compute_join_column, + _extract_predicate_names, + _validate_columns, + execute_node, +) + +from ..execution import constants + + +@execute_node.register( + ops.AsOfJoin, dd.DataFrame, dd.DataFrame, (Timedelta, type(None)) +) +def execute_asof_join(op, left, right, tolerance, **kwargs): + overlapping_columns = frozenset(left.columns) & frozenset(right.columns) + left_on, right_on = _extract_predicate_names(op.predicates) + left_by, right_by = _extract_predicate_names(op.by) + _validate_columns( + overlapping_columns, left_on, right_on, left_by, right_by + ) + + return dd.merge_asof( + left=left, + right=right, + left_on=left_on, + right_on=right_on, + left_by=left_by or None, + right_by=right_by or None, + tolerance=tolerance, + ) + + +@execute_node.register(ops.CrossJoin, dd.DataFrame, dd.DataFrame) +def execute_cross_join(op, left, right, **kwargs): + """Execute a cross join in dask. + + Notes + ----- + We create a dummy column of all :data:`True` instances and use that as the + join key. This results in the desired Cartesian product behavior guaranteed + by cross join. + + """ + # generate a unique name for the temporary join key + key = "cross_join_{}".format(ibis.util.guid()) + join_key = {key: True} + new_left = left.assign(**join_key) + new_right = right.assign(**join_key) + + # inner/outer doesn't matter because every row matches every other row + result = dd.merge( + new_left, + new_right, + how='inner', + on=key, + suffixes=constants.JOIN_SUFFIXES, + ) + + # remove the generated key + del result[key] + + return result + + +# TODO - execute_materialized_join - #2553 +@execute_node.register(ops.Join, dd.DataFrame, dd.DataFrame) +def execute_materialized_join(op, left, right, **kwargs): + op_type = type(op) + + try: + how = constants.JOIN_TYPES[op_type] + except KeyError: + raise NotImplementedError('{} not supported'.format(op_type.__name__)) + + left_op = op.left.op() + right_op = op.right.op() + + on = {left_op: [], right_op: []} + for predicate in map(operator.methodcaller('op'), op.predicates): + if not isinstance(predicate, ops.Equals): + raise TypeError( + 'Only equality join predicates supported with dask' + ) + new_left_column, left_pred_root = _compute_join_column( + predicate.left, **kwargs + ) + on[left_pred_root].append(new_left_column) + + new_right_column, right_pred_root = _compute_join_column( + predicate.right, **kwargs + ) + on[right_pred_root].append(new_right_column) + + df = dd.merge( + left, + right, + how=how, + left_on=on[left_op], + right_on=on[right_op], + suffixes=constants.JOIN_SUFFIXES, + ) + return df diff --git a/ibis/backends/dask/execution/maps.py b/ibis/backends/dask/execution/maps.py new file mode 100644 index 000000000000..003d2cbfc549 --- /dev/null +++ b/ibis/backends/dask/execution/maps.py @@ -0,0 +1,144 @@ +import collections + +import dask.dataframe as dd +import numpy as np +import pandas + +import ibis.expr.operations as ops +from ibis.backends.pandas.execution.maps import ( + execute_map_keys_series, + execute_map_value_default_dict_scalar_series, + execute_map_value_default_dict_series_scalar, + execute_map_value_default_dict_series_series, + execute_map_value_default_series_series_series, + execute_map_value_for_key_dict_series, + execute_map_value_for_key_series_scalar, + execute_map_values_series, + execute_node, + map_value_default_series_scalar_scalar, + map_value_default_series_scalar_series, + map_value_default_series_series_scalar, + safe_merge, +) + +from .util import TypeRegistrationDict, register_types_to_dispatcher + +# NOTE - to avoid dispatch ambiguities we must unregister pandas, only to +# re-register below. The ordering in which dispatches are registered is +# meaningful. See https://multiple-dispatch.readthedocs.io/en/latest/resolution.html#ambiguities # noqa E501 +# for more detail. +PANDAS_REGISTERED_TYPES = [ + ( + ops.MapValueOrDefaultForKey, + collections.abc.Mapping, + object, + pandas.Series, + ), + ( + ops.MapValueOrDefaultForKey, + collections.abc.Mapping, + pandas.Series, + object, + ), +] +for registered_type in PANDAS_REGISTERED_TYPES: + del execute_node[registered_type] + + +DASK_DISPATCH_TYPES: TypeRegistrationDict = { + ops.MapValueForKey: [ + ((dd.Series, object,), execute_map_value_for_key_series_scalar), + ( + (collections.abc.Mapping, dd.Series,), + execute_map_value_for_key_dict_series, + ), + ], + ops.MapValueOrDefaultForKey: [ + ((dd.Series, object, object), map_value_default_series_scalar_scalar), + ( + (dd.Series, object, dd.Series), + map_value_default_series_scalar_series, + ), + ( + (dd.Series, dd.Series, object), + map_value_default_series_series_scalar, + ), + ( + (dd.Series, dd.Series, dd.Series), + execute_map_value_default_series_series_series, + ), + # This never occurs but we need to register it so multipledispatch + # does not see below registrations as ambigious. See NOTE above. + ( + ( + collections.abc.Mapping, + (dd.Series, pandas.Series), + (dd.Series, pandas.Series), + ), + execute_map_value_default_dict_series_series, + ), + ( + (collections.abc.Mapping, object, (dd.Series, pandas.Series),), + execute_map_value_default_dict_scalar_series, + ), + ( + (collections.abc.Mapping, (dd.Series, pandas.Series), object,), + execute_map_value_default_dict_series_scalar, + ), + ], + ops.MapKeys: [((dd.Series,), execute_map_keys_series)], + ops.MapValues: [((dd.Series,), execute_map_values_series)], +} +register_types_to_dispatcher(execute_node, DASK_DISPATCH_TYPES) + + +@execute_node.register(ops.MapLength, dd.Series) +def execute_map_length_series(op, data, **kwargs): + return data.map(len, na_action='ignore') + + +@execute_node.register(ops.MapValueForKey, dd.Series, dd.Series) +def execute_map_value_for_key_series_series(op, data, key, **kwargs): + assert data.size == key.size, 'data.size != key.size' + return data.map( + lambda x, keyiter=iter(key.values): x.get(next(keyiter), None) + ) + + +def none_filled_dask_series(n): + dd.from_array(np.full(n, None)) + + +@execute_node.register(ops.MapValueForKey, dd.Series, type(None)) +def execute_map_value_for_key_series_none(op, data, key, **kwargs): + return none_filled_dask_series(len(data)) + + +@execute_node.register( + ops.MapConcat, (collections.abc.Mapping, type(None)), dd.Series +) +def execute_map_concat_dict_series(op, lhs, rhs, **kwargs): + if lhs is None: + return none_filled_dask_series(len(rhs)) + return rhs.map( + lambda m, lhs=lhs: safe_merge(lhs, m), meta=(rhs.name, rhs.dtype), + ) + + +@execute_node.register( + ops.MapConcat, dd.Series, (collections.abc.Mapping, type(None)) +) +def execute_map_concat_series_dict(op, lhs, rhs, **kwargs): + if rhs is None: + return none_filled_dask_series(len(lhs)) + return lhs.map( + lambda m, rhs=rhs: safe_merge(m, rhs), meta=(lhs.name, lhs.dtype), + ) + + +@execute_node.register(ops.MapConcat, dd.Series, dd.Series) +def execute_map_concat_series_series(op, lhs, rhs, **kwargs): + return lhs.map( + lambda m, rhsiter=iter(rhs.values): safe_merge(m, next(rhsiter)), + meta=(lhs.name, lhs.dtype), + ) diff --git a/ibis/backends/dask/execution/numeric.py b/ibis/backends/dask/execution/numeric.py new file mode 100644 index 000000000000..e09629211f64 --- /dev/null +++ b/ibis/backends/dask/execution/numeric.py @@ -0,0 +1,132 @@ +import collections +import decimal +import functools +import numbers + +import dask.dataframe as dd +import dask.dataframe.groupby as ddgb +import numpy as np + +import ibis.common.exceptions as com +import ibis.expr.operations as ops +from ibis.backends.pandas.core import numeric_types +from ibis.backends.pandas.execution.generic import execute_node + + +# TODO - aggregations - #2553 +@execute_node.register(ops.Arbitrary, ddgb.SeriesGroupBy, type(None)) +def execute_arbitrary_series_groupby(op, data, _, aggcontext=None, **kwargs): + how = op.how + if how is None: + how = 'first' + + if how not in {'first', 'last'}: + raise com.OperationNotDefinedError( + 'Arbitrary {!r} is not supported'.format(how) + ) + return aggcontext.agg(data, how) + + +@execute_node.register(ops.Negate, dd.Series) +def execute_series_negate(op, data, **kwargs): + return data.mul(-1) + + +# TODO - grouping - #2553 +@execute_node.register(ops.Negate, ddgb.SeriesGroupBy) +def execute_series_group_by_negate(op, data, **kwargs): + return execute_series_negate(op, data.obj, **kwargs).groupby( + data.grouper.groupings + ) + + +def call_numpy_ufunc(func, op, data, **kwargs): + if data.dtype == np.dtype(np.object_): + return data.apply( + functools.partial(execute_node, op, **kwargs), + meta=(data.name, "object"), + ) + return func(data) + + +@execute_node.register(ops.UnaryOp, dd.Series) +def execute_series_unary_op(op, data, **kwargs): + function = getattr(np, type(op).__name__.lower()) + return call_numpy_ufunc(function, op, data, **kwargs) + + +@execute_node.register((ops.Ceil, ops.Floor), dd.Series) +def execute_series_ceil(op, data, **kwargs): + return_type = np.object_ if data.dtype == np.object_ else np.int64 + func = getattr(np, type(op).__name__.lower()) + return call_numpy_ufunc(func, op, data, **kwargs).astype(return_type) + + +def vectorize_object(op, arg, *args, **kwargs): + # TODO - this works for now, but I think we can do something much better + func = np.vectorize(functools.partial(execute_node, op, **kwargs)) + out = dd.from_array(func(arg, *args), columns=arg.name) + out.index = arg.index + return out + + +@execute_node.register( + ops.Log, dd.Series, (dd.Series, numbers.Real, decimal.Decimal, type(None)) +) +def execute_series_log_with_base(op, data, base, **kwargs): + if data.dtype == np.dtype(np.object_): + return vectorize_object(op, data, base, **kwargs) + + if base is None: + return np.log(data) + return np.log(data) / np.log(base) + + +@execute_node.register(ops.Ln, dd.Series) +def execute_series_natural_log(op, data, **kwargs): + if data.dtype == np.dtype(np.object_): + return data.apply( + functools.partial(execute_node, op, **kwargs), + meta=(data.name, "object"), + ) + return np.log(data) + + +@execute_node.register( + ops.Quantile, (dd.Series, ddgb.SeriesGroupBy), numeric_types +) +def execute_series_quantile(op, data, quantile, aggcontext=None, **kwargs): + return data.quantile(q=quantile) + + +@execute_node.register(ops.MultiQuantile, dd.Series, collections.abc.Sequence) +def execute_series_quantile_sequence( + op, data, quantile, aggcontext=None, **kwargs +): + return list(data.quantile(q=quantile)) + + +# TODO - aggregations - #2553 +@execute_node.register( + ops.MultiQuantile, ddgb.SeriesGroupBy, collections.abc.Sequence +) +def execute_series_quantile_groupby( + op, data, quantile, aggcontext=None, **kwargs +): + def q(x, quantile, interpolation): + result = x.quantile(quantile, interpolation=interpolation).tolist() + res = [result for _ in range(len(x))] + return res + + result = aggcontext.agg(data, q, quantile, op.interpolation) + return result + + +@execute_node.register( + ops.Round, dd.Series, (dd.Series, np.integer, type(None), int) +) +def execute_round_series(op, data, places, **kwargs): + if data.dtype == np.dtype(np.object_): + return vectorize_object(op, data, places, **kwargs) + result = data.round(places or 0) + return result if places else result.astype('int64') diff --git a/ibis/backends/dask/execution/reductions.py b/ibis/backends/dask/execution/reductions.py new file mode 100644 index 000000000000..c493f32f63f1 --- /dev/null +++ b/ibis/backends/dask/execution/reductions.py @@ -0,0 +1,211 @@ +""" +Reduces sequences + +NOTE: This file overwrite the pandas backend registered handlers for: + +- execute_node_expr_list, +- execute_node_greatest_list, +- execute_node_least_list + +This is so we can register our handlers that transparently handle both the' +dask specific types and pandas types. This cannot be done via the +dispatcher since the top level container is a list. +""" + +import collections +import functools +import operator +from collections.abc import Sized + +import dask.array as da +import dask.dataframe as dd +import dask.dataframe.groupby as ddgb +import numpy as np +import pandas as pd +import toolz + +import ibis +import ibis.expr.operations as ops +from ibis.backends.pandas.core import execute +from ibis.backends.pandas.execution.generic import ( + execute_node, + execute_node_expr_list, + execute_node_greatest_list, + execute_node_least_list, +) + + +@toolz.curry +def promote_to_sequence(length, obj): + if isinstance(obj, dd.Series): + # we must force length computation if we have mixed types + # otherwise da.reductions can't compare arrays + return obj.to_dask_array(lengths=True) + else: + return da.from_array(np.repeat(obj, length)) + + +def pairwise_reducer(func, values): + return functools.reduce(lambda x, y: func(x, y), values) + + +def compute_row_reduction(func, value): + final_sizes = {len(x) for x in value if isinstance(x, Sized)} + if not final_sizes: + return func(value) + (final_size,) = final_sizes + arrays = list(map(promote_to_sequence(final_size), value)) + raw = pairwise_reducer(func, arrays) + return dd.from_array(raw).squeeze() + + +@execute_node.register(ops.Greatest, collections.abc.Sequence) +def dask_execute_node_greatest_list(op, value, **kwargs): + if all(type(v) != dd.Series for v in value): + return execute_node_greatest_list(op, value, **kwargs) + return compute_row_reduction(da.maximum, value) + + +@execute_node.register(ops.Least, collections.abc.Sequence) +def dask_execute_node_least_list(op, value, **kwargs): + if all(type(v) != dd.Series for v in value): + return execute_node_least_list(op, value, **kwargs) + return compute_row_reduction(da.minimum, value) + + +# TODO - aggregations - #2553 +@execute_node.register(ops.Reduction, ddgb.SeriesGroupBy, type(None)) +def execute_reduction_series_groupby( + op, data, mask, aggcontext=None, **kwargs +): + return aggcontext.agg(data, type(op).__name__.lower()) + + +def _filtered_reduction(mask, method, data): + return method(data[mask[data.index]]) + + +# TODO - grouping - #2553 +@execute_node.register(ops.Reduction, ddgb.SeriesGroupBy, ddgb.SeriesGroupBy) +def execute_reduction_series_gb_mask( + op, data, mask, aggcontext=None, **kwargs +): + method = operator.methodcaller(type(op).__name__.lower()) + return aggcontext.agg( + data, functools.partial(_filtered_reduction, mask.obj, method) + ) + + +# TODO - aggregations - #2553 +@execute_node.register(ops.Reduction, dd.Series, (dd.Series, type(None))) +def execute_reduction_series_mask(op, data, mask, aggcontext=None, **kwargs): + operand = data[mask] if mask is not None else data + return aggcontext.agg(operand, type(op).__name__.lower()) + + +# TODO - aggregations - #2553 +@execute_node.register( + (ops.CountDistinct, ops.HLLCardinality), ddgb.SeriesGroupBy, type(None) +) +def execute_count_distinct_series_groupby( + op, data, _, aggcontext=None, **kwargs +): + return aggcontext.agg(data, 'nunique') + + +# TODO - aggregations - #2553 +@execute_node.register( + (ops.CountDistinct, ops.HLLCardinality), + ddgb.SeriesGroupBy, + ddgb.SeriesGroupBy, +) +def execute_count_distinct_series_groupby_mask( + op, data, mask, aggcontext=None, **kwargs +): + return aggcontext.agg( + data, + functools.partial(_filtered_reduction, mask.obj, dd.Series.nunique), + ) + + +# TODO - aggregations - #2553 +@execute_node.register( + (ops.CountDistinct, ops.HLLCardinality), dd.Series, (dd.Series, type(None)) +) +def execute_count_distinct_series_mask( + op, data, mask, aggcontext=None, **kwargs +): + return aggcontext.agg(data[mask] if mask is not None else data, 'nunique') + + +variance_ddof = {'pop': 0, 'sample': 1} + + +# TODO - aggregations - #2553 +@execute_node.register(ops.Variance, ddgb.SeriesGroupBy, type(None)) +def execute_reduction_series_groupby_var( + op, data, _, aggcontext=None, **kwargs +): + return aggcontext.agg(data, 'var', ddof=variance_ddof[op.how]) + + +# TODO - grouping - #2553 +@execute_node.register(ops.Variance, ddgb.SeriesGroupBy, ddgb.SeriesGroupBy) +def execute_var_series_groupby_mask(op, data, mask, aggcontext=None, **kwargs): + return aggcontext.agg( + data, + lambda x, mask=mask.obj, ddof=variance_ddof[op.how]: ( + x[mask[x.index]].var(ddof=ddof) + ), + ) + + +# TODO - aggregations - #2553 +@execute_node.register(ops.Variance, dd.Series, (dd.Series, type(None))) +def execute_variance_series(op, data, mask, aggcontext=None, **kwargs): + return aggcontext.agg( + data[mask] if mask is not None else data, + 'var', + ddof=variance_ddof[op.how], + ) + + +# TODO - aggregations - #2553 +@execute_node.register(ops.StandardDev, ddgb.SeriesGroupBy, type(None)) +def execute_reduction_series_groupby_std( + op, data, _, aggcontext=None, **kwargs +): + return aggcontext.agg(data, 'std', ddof=variance_ddof[op.how]) + + +# TODO - grouping - #2553 +@execute_node.register(ops.StandardDev, ddgb.SeriesGroupBy, ddgb.SeriesGroupBy) +def execute_std_series_groupby_mask(op, data, mask, aggcontext=None, **kwargs): + return aggcontext.agg( + data, + lambda x, mask=mask.obj, ddof=variance_ddof[op.how]: ( + x[mask[x.index]].std(ddof=ddof) + ), + ) + + +# TODO - aggregations - #2553 +@execute_node.register(ops.StandardDev, dd.Series, (dd.Series, type(None))) +def execute_standard_dev_series(op, data, mask, aggcontext=None, **kwargs): + return aggcontext.agg( + data[mask] if mask is not None else data, + 'std', + ddof=variance_ddof[op.how], + ) + + +@execute_node.register(ops.ExpressionList, collections.abc.Sequence) +def dask_execute_node_expr_list(op, sequence, **kwargs): + if all(type(s) != dd.Series for s in sequence): + execute_node_expr_list(op, sequence, **kwargs) + columns = [e.get_name() for e in op.exprs] + schema = ibis.schema(list(zip(columns, (e.type() for e in op.exprs)))) + data = {col: [execute(el, **kwargs)] for col, el in zip(columns, sequence)} + return schema.apply_to( + dd.from_pandas(pd.DataFrame(data, columns=columns), npartitions=1) + ) diff --git a/ibis/backends/dask/execution/selection.py b/ibis/backends/dask/execution/selection.py new file mode 100644 index 000000000000..8964f29522d4 --- /dev/null +++ b/ibis/backends/dask/execution/selection.py @@ -0,0 +1,201 @@ +"""Dispatching code for Selection operations. +""" + +from __future__ import absolute_import + +import functools +import operator +from typing import Optional + +import dask.dataframe as dd +import numpy as np +import pandas +from toolz import concatv + +import ibis.expr.operations as ops +import ibis.expr.types as ir +from ibis.backends.pandas.execution.selection import ( + _compute_predicates, + compute_projection, + compute_projection_table_expr, + execute, + execute_node, + map_new_column_names_to_data, + remap_overlapping_column_names, +) +from ibis.expr.scope import Scope +from ibis.expr.typing import TimeContext + +from ..execution import constants + + +@compute_projection.register(ir.ScalarExpr, ops.Selection, dd.DataFrame) +def compute_projection_scalar_expr( + expr, + parent, + data, + scope: Scope = None, + timecontext: Optional[TimeContext] = None, + **kwargs, +): + name = expr._name + assert name is not None, 'Scalar selection name is None' + + op = expr.op() + parent_table_op = parent.table.op() + + data_columns = frozenset(data.columns) + + scope = scope.merge_scopes( + Scope( + { + t: map_new_column_names_to_data( + remap_overlapping_column_names( + parent_table_op, t, data_columns + ), + data, + ) + }, + timecontext, + ) + for t in op.root_tables() + ) + scalar = execute(expr, scope=scope, **kwargs) + result = pandas.Series([scalar], name=name).repeat(len(data.index)) + result.index = data.index + return dd.from_pandas(result, npartitions=data.npartitions) + + +@compute_projection.register(ir.ColumnExpr, ops.Selection, dd.DataFrame) +def compute_projection_column_expr( + expr, + parent, + data, + scope: Scope, + timecontext: Optional[TimeContext], + **kwargs, +): + result_name = getattr(expr, '_name', None) + op = expr.op() + parent_table_op = parent.table.op() + + if isinstance(op, ops.TableColumn): + # slightly faster path for simple column selection + name = op.name + + if name in data: + return data[name].rename(result_name or name) + + if not isinstance(parent_table_op, ops.Join): + raise KeyError(name) + (root_table,) = op.root_tables() + left_root, right_root = ops.distinct_roots( + parent_table_op.left, parent_table_op.right + ) + suffixes = { + left_root: constants.LEFT_JOIN_SUFFIX, + right_root: constants.RIGHT_JOIN_SUFFIX, + } + return data.loc[:, name + suffixes[root_table]].rename( + result_name or name + ) + + data_columns = frozenset(data.columns) + + scope = scope.merge_scopes( + Scope( + { + t: map_new_column_names_to_data( + remap_overlapping_column_names( + parent_table_op, t, data_columns + ), + data, + ) + }, + timecontext, + ) + for t in op.root_tables() + ) + + result = execute(expr, scope=scope, timecontext=timecontext, **kwargs) + assert result_name is not None, 'Column selection name is None' + if np.isscalar(result): + series = dd.from_array(np.repeat(result, len(data.index))) + series.name = result_name + series.index = data.index + return series + return result.rename(result_name) + + +compute_projection.register(ir.TableExpr, ops.Selection, dd.DataFrame)( + compute_projection_table_expr +) + + +# TODO - sorting - #2553 +@execute_node.register(ops.Selection, dd.DataFrame) +def execute_selection_dataframe( + op, data, scope: Scope, timecontext: Optional[TimeContext], **kwargs +): + selections = op.selections + predicates = op.predicates + sort_keys = op.sort_keys + result = data + + # Build up the individual dask structures from column expressions + if selections: + data_pieces = [] + for selection in selections: + dask_object = compute_projection( + selection, + op, + data, + scope=scope, + timecontext=timecontext, + **kwargs, + ) + data_pieces.append(dask_object) + + new_pieces = [piece for piece in data_pieces] + result = dd.concat(new_pieces, axis=1) + + if predicates: + predicates = _compute_predicates( + op.table.op(), predicates, data, scope, timecontext, **kwargs + ) + predicate = functools.reduce(operator.and_, predicates) + assert len(predicate) == len( + result + ), 'Selection predicate length does not match underlying table' + result = result.loc[predicate] + + if sort_keys: + raise NotImplementedError( + "Sorting is not implemented for the Dask backend" + ) + # result, grouping_keys, ordering_keys = util.compute_sorted_frame( + # result, + # order_by=sort_keys, + # scope=scope, + # timecontext=timecontext, + # **kwargs, + # ) + else: + grouping_keys = ordering_keys = () + + # return early if we do not have any temporary grouping or ordering columns + assert not grouping_keys, 'group by should never show up in Selection' + if not ordering_keys: + return result + + # create a sequence of columns that we need to drop + temporary_columns = pandas.Index( + concatv(grouping_keys, ordering_keys) + ).difference(data.columns) + + # no reason to call drop if we don't need to + if temporary_columns.empty: + return result + + # drop every temporary column we created for ordering or grouping + return result.drop(temporary_columns, axis=1) diff --git a/ibis/backends/dask/execution/strings.py b/ibis/backends/dask/execution/strings.py new file mode 100644 index 000000000000..9a6a7b0c0240 --- /dev/null +++ b/ibis/backends/dask/execution/strings.py @@ -0,0 +1,318 @@ +import itertools + +import dask.dataframe as dd +import dask.dataframe.groupby as ddgb +import numpy as np +import pandas +import toolz +from pandas import isnull + +import ibis +import ibis.expr.operations as ops +from ibis.backends.pandas.core import integer_types, scalar_types +from ibis.backends.pandas.execution.strings import ( + execute_node, + execute_series_join_scalar_sep, + execute_series_regex_extract, + execute_series_regex_replace, + execute_series_regex_search, + execute_series_right, + execute_series_translate_scalar_scalar, + execute_series_translate_scalar_series, + execute_series_translate_series_scalar, + execute_series_translate_series_series, + execute_string_capitalize, + execute_string_contains, + execute_string_length_series, + execute_string_like_series_string, + execute_string_lower, + execute_string_lpad, + execute_string_lstrip, + execute_string_repeat, + execute_string_reverse, + execute_string_rpad, + execute_string_rstrip, + execute_string_strip, + execute_string_upper, + execute_substring_int_int, + haystack_to_series_of_lists, +) + +from .util import TypeRegistrationDict, register_types_to_dispatcher + +DASK_DISPATCH_TYPES: TypeRegistrationDict = { + ops.StringLength: [((dd.Series,), execute_string_length_series)], + ops.Substring: [ + ( + (dd.Series, integer_types, integer_types,), + execute_substring_int_int, + ), + ], + ops.Strip: [((dd.Series,), execute_string_strip)], + ops.LStrip: [((dd.Series,), execute_string_lstrip)], + ops.RStrip: [((dd.Series,), execute_string_rstrip)], + ops.LPad: [ + ( + (dd.Series, (dd.Series,) + integer_types, (dd.Series, str),), + execute_string_lpad, + ), + ], + ops.RPad: [ + ( + (dd.Series, (dd.Series,) + integer_types, (dd.Series, str),), + execute_string_rpad, + ), + ], + ops.Reverse: [((dd.Series,), execute_string_reverse)], + ops.Lowercase: [((dd.Series,), execute_string_lower)], + ops.Uppercase: [((dd.Series,), execute_string_upper)], + ops.Capitalize: [((dd.Series,), execute_string_capitalize)], + ops.Repeat: [ + ((dd.Series, (dd.Series,) + integer_types), execute_string_repeat), + ], + ops.StringFind: [ + ( + ( + dd.Series, + (dd.Series, str), + (dd.Series, type(None)) + integer_types, + (dd.Series, type(None)) + integer_types, + ), + execute_string_contains, + ) + ], + ops.StringSQLLike: [ + ( + (dd.Series, str, (str, type(None)),), + execute_string_like_series_string, + ), + ], + ops.RegexSearch: [((dd.Series, str,), execute_series_regex_search)], + ops.RegexExtract: [ + ( + (dd.Series, (dd.Series, str), integer_types), + execute_series_regex_extract, + ), + ], + ops.RegexReplace: [ + ((dd.Series, str, str,), execute_series_regex_replace), + ], + ops.Translate: [ + ( + (dd.Series, dd.Series, dd.Series), + execute_series_translate_series_series, + ), + ((dd.Series, dd.Series, str), execute_series_translate_series_scalar), + ((dd.Series, str, dd.Series), execute_series_translate_scalar_series), + ((dd.Series, str, str), execute_series_translate_scalar_scalar), + ], + ops.StrRight: [((dd.Series, integer_types), execute_series_right)], + ops.StringJoin: [ + (((dd.Series, str), list), execute_series_join_scalar_sep), + ], +} +register_types_to_dispatcher(execute_node, DASK_DISPATCH_TYPES) + + +@execute_node.register(ops.Substring, dd.Series, dd.Series, integer_types) +def execute_substring_series_int(op, data, start, length, **kwargs): + return execute_substring_series_series( + op, data, start, dd.from_array(np.repeat(length, len(start))), **kwargs + ) + + +@execute_node.register(ops.Substring, dd.Series, integer_types, dd.Series) +def execute_string_substring_int_series(op, data, start, length, **kwargs): + return execute_substring_series_series( + op, + data, + dd.from_array(np.repeat(start, len(length))), + length, + **kwargs, + ) + + +# TODO - substring - #2553 +@execute_node.register(ops.Substring, dd.Series, dd.Series, dd.Series) +def execute_substring_series_series(op, data, start, length, **kwargs): + end = start + length + + # TODO - this is broken + def iterate( + value, start_iter=start.iteritems(), end_iter=end.iteritems(), + ): + _, begin = next(start_iter) + _, end = next(end_iter) + if (begin is not None and isnull(begin)) or ( + end is not None and isnull(end) + ): + return None + return value[begin:end] + + return data.map(iterate) + + +# TODO - grouping - #2553 +@execute_node.register(ops.StringSQLLike, ddgb.SeriesGroupBy, str, str) +def execute_string_like_series_groupby_string( + op, data, pattern, escape, **kwargs +): + return execute_string_like_series_string( + op, data.obj, pattern, escape, **kwargs + ).groupby(data.grouper.groupings) + + +# TODO - aggregations - #2553 +@execute_node.register( + ops.GroupConcat, dd.Series, str, (dd.Series, type(None)) +) +def execute_group_concat_series_mask( + op, data, sep, mask, aggcontext=None, **kwargs +): + return aggcontext.agg( + data[mask] if mask is not None else data, + lambda series, sep=sep: sep.join(series.values), + ) + + +# TODO - grouping - #2553 +@execute_node.register(ops.GroupConcat, ddgb.SeriesGroupBy, str, type(None)) +def execute_group_concat_series_gb( + op, data, sep, _, aggcontext=None, **kwargs +): + custom_group_concat = dd.Aggregation( + name='custom_group_concat', + chunk=lambda s: s.apply(list), + agg=lambda s0: s0.apply( + lambda chunks: sep.join( + str(s) for s in itertools.chain.from_iterable(chunks) + ) + ), + ) + return data.agg(custom_group_concat) + + +# TODO - grouping - #2553 +@execute_node.register( + ops.GroupConcat, ddgb.SeriesGroupBy, str, ddgb.SeriesGroupBy +) +def execute_group_concat_series_gb_mask( + op, data, sep, mask, aggcontext=None, **kwargs +): + def method(series, sep=sep): + return sep.join(series.values.astype(str)) + + return aggcontext.agg( + data, + lambda data, mask=mask.obj, method=method: method( + data[mask[data.index]] + ), + ) + + +@execute_node.register(ops.StringAscii, dd.Series) +def execute_string_ascii(op, data, **kwargs): + output_meta = pandas.Series([], dtype=np.dtype('int32'), name=data.name) + return data.map(ord, meta=output_meta) + + +# TODO - grouping - #2553 +@execute_node.register(ops.StringAscii, ddgb.SeriesGroupBy) +def execute_string_ascii_group_by(op, data, **kwargs): + return execute_string_ascii(op, data, **kwargs).groupby( + data.grouper.groupings + ) + + +# TODO - grouping - #2553 +@execute_node.register(ops.RegexSearch, ddgb.SeriesGroupBy, str) +def execute_series_regex_search_gb(op, data, pattern, **kwargs): + return execute_series_regex_search( + op, data, getattr(pattern, 'obj', pattern), **kwargs + ).groupby(data.grouper.groupings) + + +# TODO - grouping - #2553 +@execute_node.register( + ops.RegexExtract, ddgb.SeriesGroupBy, str, integer_types +) +def execute_series_regex_extract_gb(op, data, pattern, index, **kwargs): + return execute_series_regex_extract( + op, data.obj, pattern, index, **kwargs + ).groupby(data.grouper.groupings) + + +# TODO - grouping - #2553 +@execute_node.register(ops.RegexReplace, ddgb.SeriesGroupBy, str, str) +def execute_series_regex_replace_gb(op, data, pattern, replacement, **kwargs): + return execute_series_regex_replace( + data.obj, pattern, replacement, **kwargs + ).groupby(data.grouper.groupings) + + +# TODO - grouping - #2553 +@execute_node.register(ops.StrRight, ddgb.SeriesGroupBy, integer_types) +def execute_series_right_gb(op, data, nchars, **kwargs): + return execute_series_right(op, data.obj, nchars).groupby( + data.grouper.groupings + ) + + +def haystack_to_dask_series_of_lists(haystack, index=None): + pieces = haystack_to_series_of_lists(haystack, index) + return dd.from_pandas(pieces, npartitions=1) + + +@execute_node.register(ops.FindInSet, dd.Series, list) +def execute_series_find_in_set(op, needle, haystack, **kwargs): + def find_in_set(index, elements): + return ibis.util.safe_index(elements, index) + + return needle.apply(find_in_set, args=(haystack,)) + + +# TODO - grouping - #2553 +@execute_node.register(ops.FindInSet, ddgb.SeriesGroupBy, list) +def execute_series_group_by_find_in_set(op, needle, haystack, **kwargs): + pieces = [getattr(piece, 'obj', piece) for piece in haystack] + return execute_series_find_in_set( + op, needle.obj, pieces, **kwargs + ).groupby(needle.grouper.groupings) + + +# TODO we need this version not pandas +@execute_node.register(ops.FindInSet, scalar_types, list) +def execute_string_group_by_find_in_set(op, needle, haystack, **kwargs): + # `list` could contain series, series groupbys, or scalars + # mixing series and series groupbys is not allowed + series_in_haystack = [ + type(piece) + for piece in haystack + if isinstance(piece, (dd.Series, ddgb.SeriesGroupBy)) + ] + + if not series_in_haystack: + return ibis.util.safe_index(haystack, needle) + + try: + (collection_type,) = frozenset(map(type, series_in_haystack)) + except ValueError: + raise ValueError('Mixing Series and ddgb.SeriesGroupBy is not allowed') + + pieces = haystack_to_dask_series_of_lists( + [getattr(piece, 'obj', piece) for piece in haystack] + ) + + result = pieces.map(toolz.flip(ibis.util.safe_index)(needle)) + if issubclass(collection_type, dd.Series): + return result + + assert issubclass(collection_type, ddgb.SeriesGroupBy) + + return result.groupby( + toolz.first( + piece.grouper.groupings + for piece in haystack + if hasattr(piece, 'grouper') + ) + ) diff --git a/ibis/backends/dask/execution/structs.py b/ibis/backends/dask/execution/structs.py new file mode 100644 index 000000000000..27d5d4e7c978 --- /dev/null +++ b/ibis/backends/dask/execution/structs.py @@ -0,0 +1,29 @@ +"""Dask backend execution of struct fields and literals.""" + +import operator + +import dask.dataframe as dd +import dask.dataframe.groupby as ddgb + +import ibis.expr.operations as ops +from ibis.backends.pandas.execution.structs import execute_node + + +@execute_node.register(ops.StructField, dd.Series) +def execute_node_struct_field_series(op, data, **kwargs): + field = op.field + # TODO This meta is not necessarily right + return data.map( + operator.itemgetter(field), meta=(data.name, data.dtype) + ).rename(field) + + +# TODO - grouping - #2553 +@execute_node.register(ops.StructField, ddgb.SeriesGroupBy) +def execute_node_struct_field_series_group_by(op, data, **kwargs): + field = op.field + return ( + data.obj.map(operator.itemgetter(field)) + .rename(field) + .groupby(data.grouper.groupings) + ) diff --git a/ibis/backends/dask/execution/temporal.py b/ibis/backends/dask/execution/temporal.py new file mode 100644 index 000000000000..1f9c2e5cb446 --- /dev/null +++ b/ibis/backends/dask/execution/temporal.py @@ -0,0 +1,177 @@ +import datetime + +import dask.array as da +import dask.dataframe as dd +import dask.dataframe.groupby as ddgb +import numpy as np +from pandas import Timedelta + +import ibis.expr.datatypes as dt +import ibis.expr.operations as ops +from ibis.backends.pandas.core import ( + date_types, + integer_types, + numeric_types, + timedelta_types, + timestamp_types, +) +from ibis.backends.pandas.execution.temporal import ( + day_name, + execute_cast_integer_to_interval_series, + execute_date_add, + execute_date_sub_diff, + execute_day_of_week_index_series, + execute_day_of_week_name_series, + execute_epoch_seconds, + execute_extract_millisecond_series, + execute_extract_timestamp_field_series, + execute_interval_add_multiply_delta_series, + execute_interval_from_integer_series, + execute_interval_multiply_fdiv_series_numeric, + execute_node, + execute_strftime_series_str, + execute_timestamp_add_datetime_series, + execute_timestamp_date, + execute_timestamp_diff_series_datetime, + execute_timestamp_diff_sub_datetime_series, + execute_timestamp_diff_sub_series_series, + execute_timestamp_from_unix, + execute_timestamp_interval_add_series_delta, + execute_timestamp_interval_add_series_series, + execute_timestamp_sub_series_timedelta, +) + +from .util import TypeRegistrationDict, register_types_to_dispatcher + +DASK_DISPATCH_TYPES: TypeRegistrationDict = { + ops.Cast: [ + ((dd.Series, dt.Interval), execute_cast_integer_to_interval_series) + ], + ops.Strftime: [((dd.Series, str), execute_strftime_series_str)], + ops.TimestampFromUNIX: [ + (((dd.Series,) + integer_types), execute_timestamp_from_unix) + ], + ops.ExtractTemporalField: [ + ((dd.Series,), execute_extract_timestamp_field_series) + ], + ops.ExtractMillisecond: [ + ((dd.Series,), execute_extract_millisecond_series) + ], + ops.ExtractEpochSeconds: [((dd.Series,), execute_epoch_seconds)], + ops.IntervalFromInteger: [ + ((dd.Series,), execute_interval_from_integer_series) + ], + ops.IntervalAdd: [ + ( + (timedelta_types, dd.Series), + execute_interval_add_multiply_delta_series, + ), + ( + (dd.Series, timedelta_types), + execute_timestamp_interval_add_series_delta, + ), + ((dd.Series, dd.Series), execute_timestamp_interval_add_series_series), + ], + ops.IntervalSubtract: [ + ((dd.Series, dd.Series), execute_timestamp_diff_sub_series_series) + ], + ops.IntervalMultiply: [ + ( + (timedelta_types, numeric_types + (dd.Series,)), + execute_interval_add_multiply_delta_series, + ), + ( + (dd.Series, numeric_types + (dd.Series,)), + execute_interval_multiply_fdiv_series_numeric, + ), + ], + ops.IntervalFloorDivide: [ + ( + ((Timedelta, dd.Series), numeric_types + (dd.Series,),), + execute_interval_multiply_fdiv_series_numeric, + ) + ], + ops.TimestampAdd: [ + ((timestamp_types, dd.Series), execute_timestamp_add_datetime_series), + ( + (dd.Series, timedelta_types), + execute_timestamp_interval_add_series_delta, + ), + ((dd.Series, dd.Series), execute_timestamp_interval_add_series_series), + ], + ops.TimestampSub: [ + ((dd.Series, timedelta_types), execute_timestamp_sub_series_timedelta), + ( + (timestamp_types, dd.Series), + execute_timestamp_diff_sub_datetime_series, + ), + ], + (ops.TimestampDiff, ops.TimestampSub): [ + ((dd.Series, dd.Series), execute_timestamp_diff_sub_series_series) + ], + ops.TimestampDiff: [ + ((dd.Series, timestamp_types), execute_timestamp_diff_series_datetime), + ( + (timestamp_types, dd.Series), + execute_timestamp_diff_sub_datetime_series, + ), + ], + ops.DayOfWeekIndex: [((dd.Series,), execute_day_of_week_index_series)], + ops.DayOfWeekName: [((dd.Series,), execute_day_of_week_name_series)], + ops.Date: [((dd.Series,), execute_timestamp_date)], + ops.DateAdd: [ + ((dd.Series, timedelta_types), execute_date_add), + ((timedelta_types, dd.Series), execute_date_add), + ((dd.Series, dd.Series), execute_date_add), + ((date_types, dd.Series), execute_date_add), + ((dd.Series, date_types), execute_date_add), + ], + ops.DateSub: [ + ((date_types, dd.Series), execute_date_sub_diff), + ((dd.Series, dd.Series), execute_date_sub_diff), + ((dd.Series, timedelta_types), execute_date_sub_diff), + ], + ops.DateDiff: [ + ((date_types, dd.Series), execute_date_sub_diff), + ((dd.Series, dd.Series), execute_date_sub_diff), + ((dd.Series, date_types), execute_date_sub_diff), + ], +} +register_types_to_dispatcher(execute_node, DASK_DISPATCH_TYPES) + + +@execute_node.register( + ops.BetweenTime, + dd.Series, + (dd.Series, str, datetime.time), + (dd.Series, str, datetime.time), +) +def execute_between_time(op, data, lower, upper, **kwargs): + # TODO - Can this be done better? + indexer = ( + (data.dt.time.astype(str) >= lower) + & (data.dt.time.astype(str) <= upper) + ).to_dask_array(True) + + result = da.zeros(len(data), dtype=np.bool_) + result[indexer] = True + return dd.from_array(result) + + +@execute_node.register((ops.TimestampTruncate, ops.DateTruncate), dd.Series) +def execute_timestamp_truncate(op, data, **kwargs): + dtype = 'datetime64[{}]'.format(op.unit) + return data.astype(dtype) + + +# TODO - grouping - #2553 +@execute_node.register(ops.DayOfWeekIndex, ddgb.SeriesGroupBy) +def execute_day_of_week_index_series_group_by(op, data, **kwargs): + groupings = data.grouper.groupings + return data.obj.dt.dayofweek.astype(np.int16).groupby(groupings) + + +# TODO - grouping - #2553 +@execute_node.register(ops.DayOfWeekName, ddgb.SeriesGroupBy) +def execute_day_of_week_name_series_group_by(op, data, **kwargs): + return day_name(data.obj.dt).groupby(data.grouper.groupings) diff --git a/ibis/backends/dask/execution/util.py b/ibis/backends/dask/execution/util.py new file mode 100644 index 000000000000..e0821c116f9e --- /dev/null +++ b/ibis/backends/dask/execution/util.py @@ -0,0 +1,20 @@ +from typing import Callable, Dict, List, Tuple, Type + +import ibis.expr.operations as ops +from ibis.backends.pandas.trace import TraceTwoLevelDispatcher + +DispatchRule = Tuple[Tuple[Type], Callable] + +TypeRegistrationDict = Dict[ops.Node, List[DispatchRule]] + + +def register_types_to_dispatcher( + dispatcher: TraceTwoLevelDispatcher, types: TypeRegistrationDict +): + """ + Many dask operations utilize the functions defined in the pandas backend + without modification. This function helps perform registrations in bulk + """ + for ibis_op, registration_list in types.items(): + for types_to_register, fn in registration_list: + dispatcher.register(ibis_op, *types_to_register)(fn) diff --git a/ibis/backends/dask/tests/execution/conftest.py b/ibis/backends/dask/tests/execution/conftest.py index 9957eb612da2..576244e395d1 100644 --- a/ibis/backends/dask/tests/execution/conftest.py +++ b/ibis/backends/dask/tests/execution/conftest.py @@ -7,9 +7,9 @@ import pandas as pd import pytest -# import ibis.expr.datatypes as dt +import ibis.expr.datatypes as dt -# from ... import connect +from ... import connect @pytest.fixture(scope='module') @@ -84,7 +84,7 @@ def batting_df(): elif not os.path.isfile(path): pytest.skip('{} is not a file'.format(path)) - df = dd.read_csv(path, index_col=None, sep=',') + df = dd.read_csv(path, assume_missing=True) num_rows = int(0.01 * len(df)) return df.iloc[30 : 30 + num_rows].reset_index(drop=True) @@ -99,7 +99,7 @@ def awards_players_df(): elif not os.path.isfile(path): pytest.skip('{} is not a file'.format(path)) - return dd.read_csv(path, index_col=None, sep=',') + return dd.read_csv(path, assume_missing=True) @pytest.fixture(scope='module') @@ -190,135 +190,135 @@ def time_keyed_df2(): return dd.from_pandas(pandas_df, npartitions=1) -# @pytest.fixture(scope='module') -# def client( -# df, -# df1, -# df2, -# df3, -# time_df1, -# time_df2, -# time_df3, -# time_keyed_df1, -# time_keyed_df2, -# intersect_df2, -# ): -# return connect( -# dict( -# df=df, -# df1=df1, -# df2=df2, -# df3=df3, -# left=df1, -# right=df2, -# time_df1=time_df1, -# time_df2=time_df2, -# time_df3=time_df3, -# time_keyed_df1=time_keyed_df1, -# time_keyed_df2=time_keyed_df2, -# intersect_df2=intersect_df2, -# ) -# ) +@pytest.fixture(scope='module') +def client( + df, + df1, + df2, + df3, + time_df1, + time_df2, + time_df3, + time_keyed_df1, + time_keyed_df2, + intersect_df2, +): + return connect( + dict( + df=df, + df1=df1, + df2=df2, + df3=df3, + left=df1, + right=df2, + time_df1=time_df1, + time_df2=time_df2, + time_df3=time_df3, + time_keyed_df1=time_keyed_df1, + time_keyed_df2=time_keyed_df2, + intersect_df2=intersect_df2, + ) + ) -# @pytest.fixture(scope='module') -# def df3(): -# pandas_df = pd.DataFrame( -# { -# 'key': list('ac'), -# 'other_value': [4.0, 6.0], -# 'key2': list('ae'), -# 'key3': list('fe'), -# } -# ) -# return dd.from_pandas(pandas_df, npartitions=1) +@pytest.fixture(scope='module') +def df3(): + pandas_df = pd.DataFrame( + { + 'key': list('ac'), + 'other_value': [4.0, 6.0], + 'key2': list('ae'), + 'key3': list('fe'), + } + ) + return dd.from_pandas(pandas_df, npartitions=1) -# t_schema = { -# 'decimal': dt.Decimal(4, 3), -# 'array_of_float64': dt.Array(dt.double), -# 'array_of_int64': dt.Array(dt.int64), -# 'array_of_strings': dt.Array(dt.string), -# 'map_of_strings_integers': dt.Map(dt.string, dt.int64), -# 'map_of_integers_strings': dt.Map(dt.int64, dt.string), -# 'map_of_complex_values': dt.Map(dt.string, dt.Array(dt.int64)), -# } +t_schema = { + 'decimal': dt.Decimal(4, 3), + 'array_of_float64': dt.Array(dt.double), + 'array_of_int64': dt.Array(dt.int64), + 'array_of_strings': dt.Array(dt.string), + 'map_of_strings_integers': dt.Map(dt.string, dt.int64), + 'map_of_integers_strings': dt.Map(dt.int64, dt.string), + 'map_of_complex_values': dt.Map(dt.string, dt.Array(dt.int64)), +} -# @pytest.fixture(scope='module') -# def t(client): -# return client.table('df', schema=t_schema) +@pytest.fixture(scope='module') +def t(client): + return client.table('df', schema=t_schema) -# @pytest.fixture(scope='module') -# def lahman(batting_df, awards_players_df): -# return connect( -# {'batting': batting_df, 'awards_players': awards_players_df} -# ) +@pytest.fixture(scope='module') +def lahman(batting_df, awards_players_df): + return connect( + {'batting': batting_df, 'awards_players': awards_players_df} + ) -# @pytest.fixture(scope='module') -# def left(client): -# return client.table('left') +@pytest.fixture(scope='module') +def left(client): + return client.table('left') -# @pytest.fixture(scope='module') -# def right(client): -# return client.table('right') +@pytest.fixture(scope='module') +def right(client): + return client.table('right') -# @pytest.fixture(scope='module') -# def time_left(client): -# return client.table('time_df1') +@pytest.fixture(scope='module') +def time_left(client): + return client.table('time_df1') -# @pytest.fixture(scope='module') -# def time_right(client): -# return client.table('time_df2') +@pytest.fixture(scope='module') +def time_right(client): + return client.table('time_df2') -# @pytest.fixture(scope='module') -# def time_table(client): -# return client.table('time_df3') +@pytest.fixture(scope='module') +def time_table(client): + return client.table('time_df3') -# @pytest.fixture(scope='module') -# def time_keyed_left(client): -# return client.table('time_keyed_df1') +@pytest.fixture(scope='module') +def time_keyed_left(client): + return client.table('time_keyed_df1') -# @pytest.fixture(scope='module') -# def time_keyed_right(client): -# return client.table('time_keyed_df2') +@pytest.fixture(scope='module') +def time_keyed_right(client): + return client.table('time_keyed_df2') -# @pytest.fixture(scope='module') -# def batting(lahman): -# return lahman.table('batting') +@pytest.fixture(scope='module') +def batting(lahman): + return lahman.table('batting') -# @pytest.fixture(scope='module') -# def awards_players(lahman): -# return lahman.table('awards_players') +@pytest.fixture(scope='module') +def awards_players(lahman): + return lahman.table('awards_players') -# @pytest.fixture(scope='module') -# def sel_cols(batting): -# cols = batting.columns -# start, end = cols.index('AB'), cols.index('H') + 1 -# return ['playerID', 'yearID', 'teamID', 'G'] + cols[start:end] +@pytest.fixture(scope='module') +def sel_cols(batting): + cols = batting.columns + start, end = cols.index('AB'), cols.index('H') + 1 + return ['playerID', 'yearID', 'teamID', 'G'] + cols[start:end] -# @pytest.fixture(scope='module') -# def players_base(batting, sel_cols): -# return batting[sel_cols].sort_by(sel_cols[:3]) +@pytest.fixture(scope='module') +def players_base(batting, sel_cols): + return batting[sel_cols].sort_by(sel_cols[:3]) -# @pytest.fixture(scope='module') -# def players(players_base): -# return players_base.groupby('playerID') +@pytest.fixture(scope='module') +def players(players_base): + return players_base.groupby('playerID') -# @pytest.fixture(scope='module') -# def players_df(players_base): -# return players_base.execute().reset_index(drop=True) +@pytest.fixture(scope='module') +def players_df(players_base): + return players_base.execute().reset_index(drop=True) diff --git a/ibis/backends/dask/tests/execution/test_arrays.py b/ibis/backends/dask/tests/execution/test_arrays.py new file mode 100644 index 000000000000..9bdff55a48a9 --- /dev/null +++ b/ibis/backends/dask/tests/execution/test_arrays.py @@ -0,0 +1,245 @@ +import operator + +import dask.dataframe as dd +import pandas as pd +import pytest +from dask.dataframe.utils import tm +from pytest import param + +import ibis +from ibis.common.exceptions import IbisTypeError + + +def test_array_length(t, df): + expr = t.projection( + [ + t.array_of_float64.length().name('array_of_float64_length'), + t.array_of_int64.length().name('array_of_int64_length'), + t.array_of_strings.length().name('array_of_strings_length'), + ] + ) + result = expr.execute() + expected = dd.from_pandas( + pd.DataFrame( + { + 'array_of_float64_length': [2, 1, 0], + 'array_of_int64_length': [2, 0, 1], + 'array_of_strings_length': [2, 0, 1], + } + ), + npartitions=1, + ) + + tm.assert_frame_equal(result.compute(), expected.compute()) + + +def test_array_length_scalar(client): + raw_value = [1, 2, 4] + value = ibis.literal(raw_value) + expr = value.length() + result = client.execute(expr) + expected = len(raw_value) + assert result == expected + + +def test_array_collect(t, df): + expr = t.group_by(t.dup_strings).aggregate( + collected=t.float64_with_zeros.collect() + ) + result = expr.execute() + expected = ( + df.groupby('dup_strings') + .float64_with_zeros.apply(list) + .reset_index() + .rename(columns={'float64_with_zeros': 'collected'}) + ) + tm.assert_frame_equal(result.compute(), expected.compute()) + + +@pytest.mark.xfail( + raises=NotImplementedError, reason='TODO - windowing - #2553' +) +def test_array_collect_rolling_partitioned(t, df): + window = ibis.trailing_window(1, order_by=t.plain_int64) + colexpr = t.plain_float64.collect().over(window) + expr = t['dup_strings', 'plain_int64', colexpr.name('collected')] + result = expr.execute() + expected = dd.from_pandas( + pd.DataFrame( + { + 'dup_strings': ['d', 'a', 'd'], + 'plain_int64': [1, 2, 3], + 'collected': [[4.0], [4.0, 5.0], [5.0, 6.0]], + } + ), + npartitions=1, + )[expr.columns] + tm.assert_frame_equal(result.compute(), expected.compute()) + + +@pytest.mark.xfail(raises=IbisTypeError, reason='Not sure if this should work') +def test_array_collect_scalar(client): + raw_value = 'abcd' + value = ibis.literal(raw_value) + expr = value.collect() + result = client.execute(expr) + expected = [raw_value] + assert result == expected + + +@pytest.mark.parametrize( + ['start', 'stop'], + [ + (1, 3), + (1, 1), + (2, 3), + (2, 5), + (None, 3), + (None, None), + (3, None), + # negative slices are not supported + param( + -3, + None, + marks=pytest.mark.xfail( + raises=ValueError, reason='Negative slicing not supported' + ), + ), + param( + None, + -3, + marks=pytest.mark.xfail( + raises=ValueError, reason='Negative slicing not supported' + ), + ), + param( + -3, + -1, + marks=pytest.mark.xfail( + raises=ValueError, reason='Negative slicing not supported' + ), + ), + ], +) +def test_array_slice(t, df, start, stop): + expr = t.array_of_strings[start:stop] + result = expr.execute() + slicer = operator.itemgetter(slice(start, stop)) + expected = df.array_of_strings.apply(slicer) + tm.assert_series_equal(result.compute(), expected.compute()) + + +@pytest.mark.parametrize( + ['start', 'stop'], + [ + (1, 3), + (1, 1), + (2, 3), + (2, 5), + (None, 3), + (None, None), + (3, None), + # negative slices are not supported + param( + -3, + None, + marks=pytest.mark.xfail( + raises=ValueError, reason='Negative slicing not supported' + ), + ), + param( + None, + -3, + marks=pytest.mark.xfail( + raises=ValueError, reason='Negative slicing not supported' + ), + ), + param( + -3, + -1, + marks=pytest.mark.xfail( + raises=ValueError, reason='Negative slicing not supported' + ), + ), + ], +) +def test_array_slice_scalar(client, start, stop): + raw_value = [-11, 42, 10] + value = ibis.literal(raw_value) + expr = value[start:stop] + result = client.execute(expr) + expected = raw_value[start:stop] + assert result == expected + + +@pytest.mark.parametrize('index', [1, 3, 4, 11, -11]) +def test_array_index(t, df, index): + expr = t[t.array_of_float64[index].name('indexed')] + result = expr.execute() + expected = dd.from_pandas( + pd.DataFrame( + { + 'indexed': df.array_of_float64.apply( + lambda x: x[index] if -len(x) <= index < len(x) else None + ) + } + ), + npartitions=1, + ) + tm.assert_frame_equal(result.compute(), expected.compute()) + + +@pytest.mark.parametrize('index', [1, 3, 4, 11]) +def test_array_index_scalar(client, index): + raw_value = [-10, 1, 2, 42] + value = ibis.literal(raw_value) + expr = value[index] + result = client.execute(expr) + expected = raw_value[index] if index < len(raw_value) else None + assert result == expected + + +@pytest.mark.parametrize('n', [1, 3, 4, 7, -2]) # negative returns empty list +@pytest.mark.parametrize('mul', [lambda x, n: x * n, lambda x, n: n * x]) +def test_array_repeat(t, df, n, mul): + expr = t.projection([mul(t.array_of_strings, n).name('repeated')]) + result = expr.execute() + expected = dd.from_pandas( + pd.DataFrame({'repeated': df.array_of_strings * n}), npartitions=1, + ) + tm.assert_frame_equal(result.compute(), expected.compute()) + + +@pytest.mark.parametrize('n', [1, 3, 4, 7, -2]) # negative returns empty list +@pytest.mark.parametrize('mul', [lambda x, n: x * n, lambda x, n: n * x]) +def test_array_repeat_scalar(client, n, mul): + raw_array = [1, 2] + array = ibis.literal(raw_array) + expr = mul(array, n) + result = client.execute(expr) + expected = mul(raw_array, n) + assert result == expected + + +@pytest.mark.parametrize('op', [lambda x, y: x + y, lambda x, y: y + x]) +def test_array_concat(t, df, op): + x = t.array_of_float64.cast('array') + y = t.array_of_strings + expr = op(x, y) + result = expr.execute() + expected = op( + df.array_of_float64.apply(lambda x: list(map(str, x))), + df.array_of_strings, + ) + tm.assert_series_equal(result.compute(), expected.compute()) + + +@pytest.mark.parametrize('op', [lambda x, y: x + y, lambda x, y: y + x]) +def test_array_concat_scalar(client, op): + raw_left = [1, 2, 3] + raw_right = [3, 4] + left = ibis.literal(raw_left) + right = ibis.literal(raw_right) + expr = op(left, right) + result = client.execute(expr) + assert result == op(raw_left, raw_right) diff --git a/ibis/backends/dask/tests/execution/test_cast.py b/ibis/backends/dask/tests/execution/test_cast.py new file mode 100644 index 000000000000..aaab70615043 --- /dev/null +++ b/ibis/backends/dask/tests/execution/test_cast.py @@ -0,0 +1,173 @@ +import decimal + +import pytest +from dask.dataframe.utils import tm # noqa: E402 +from pandas import Timestamp +from pytest import param + +import ibis +import ibis.expr.datatypes as dt # noqa: E402 + +from ... import execute + +pytestmark = pytest.mark.dask + + +@pytest.mark.parametrize('from_', ['plain_float64', 'plain_int64']) +@pytest.mark.parametrize( + ('to', 'expected'), + [ + ('float16', 'float16'), + ('float32', 'float32'), + ('float64', 'float64'), + ('double', 'float64'), + ('float', 'float32'), + ('int8', 'int8'), + ('int16', 'int16'), + ('int32', 'int32'), + ('int64', 'int64'), + ('string', 'object'), + ], +) +def test_cast_numeric(t, df, from_, to, expected): + c = t[from_].cast(to) + result = c.execute() + assert str(result.dtype) == expected + + +@pytest.mark.parametrize('from_', ['float64_as_strings', 'int64_as_strings']) +@pytest.mark.parametrize( + ('to', 'expected'), [('double', 'float64'), ('string', 'object')] +) +def test_cast_string(t, df, from_, to, expected): + c = t[from_].cast(to) + result = c.execute() + assert str(result.dtype) == expected + + +@pytest.mark.parametrize( + ('to', 'expected'), + [ + ('string', 'object'), + ('int64', 'int64'), + param('double', 'float64', marks=pytest.mark.xfail(raises=TypeError)), + ( + dt.Timestamp('America/Los_Angeles'), + 'datetime64[ns, America/Los_Angeles]', + ), + ( + "timestamp('America/Los_Angeles')", + 'datetime64[ns, America/Los_Angeles]', + ), + ], +) +@pytest.mark.parametrize( + 'column', + ['plain_datetimes_naive', 'plain_datetimes_ny', 'plain_datetimes_utc'], +) +def test_cast_timestamp_column(t, df, column, to, expected): + c = t[column].cast(to) + result = c.execute() + assert str(result.dtype) == expected + + +@pytest.mark.parametrize( + ('to', 'expected'), + [ + ('string', str), + ('int64', lambda x: x.value), + param( + 'double', + float, + marks=pytest.mark.xfail(raises=NotImplementedError), + ), + ( + dt.Timestamp('America/Los_Angeles'), + lambda x: x.tz_localize('America/Los_Angeles'), + ), + ], +) +def test_cast_timestamp_scalar_naive(to, expected): + literal_expr = ibis.literal(Timestamp('now')) + value = literal_expr.cast(to) + result = execute(value) + raw = execute(literal_expr) + assert result == expected(raw) + + +@pytest.mark.parametrize( + ('to', 'expected'), + [ + ('string', str), + ('int64', lambda x: x.value), + param( + 'double', + float, + marks=pytest.mark.xfail(raises=NotImplementedError), + ), + ( + dt.Timestamp('America/Los_Angeles'), + lambda x: x.tz_convert('America/Los_Angeles'), + ), + ], +) +@pytest.mark.parametrize('tz', ['UTC', 'America/New_York']) +def test_cast_timestamp_scalar(to, expected, tz): + literal_expr = ibis.literal(Timestamp('now').tz_localize(tz)) + value = literal_expr.cast(to) + result = execute(value) + raw = execute(literal_expr) + assert result == expected(raw) + + +def test_timestamp_with_timezone_is_inferred_correctly(t, df): + assert t.plain_datetimes_naive.type().equals(dt.timestamp) + assert t.plain_datetimes_ny.type().equals(dt.Timestamp('America/New_York')) + assert t.plain_datetimes_utc.type().equals(dt.Timestamp('UTC')) + + +@pytest.mark.parametrize( + 'column', + ['plain_datetimes_naive', 'plain_datetimes_ny', 'plain_datetimes_utc'], +) +def test_cast_date(t, df, column): + expr = t[column].cast('date') + result = expr.execute() + expected = df[column].dt.normalize() + tm.assert_series_equal(result.compute(), expected.compute()) + + +@pytest.mark.parametrize('type', [dt.Decimal(9, 0), dt.Decimal(12, 3)]) +def test_cast_to_decimal(t, df, type): + expr = t.float64_as_strings.cast(type) + result = expr.execute() + context = decimal.Context(prec=type.precision) + expected = df.float64_as_strings.apply( + lambda x: context.create_decimal(x).quantize( + decimal.Decimal( + '{}.{}'.format( + '0' * (type.precision - type.scale), '0' * type.scale + ) + ) + ), + meta=("float64_as_strings", "object"), + ) + tm.assert_series_equal(result.compute(), expected.compute()) + assert all( + abs(element.as_tuple().exponent) == type.scale + for element in result.compute().values + ) + assert all( + 1 <= len(element.as_tuple().digits) <= type.precision + for element in result.compute().values + ) + + +@pytest.mark.parametrize( + 'column', ['plain_int64', 'dup_strings', 'dup_ints', 'strings_with_nulls'], +) +def test_cast_to_category(t, df, column): + test = t[column].cast('category').execute() + tm.assert_series_equal( + test.compute(), df[column].astype('category').compute() + ) diff --git a/ibis/backends/dask/tests/execution/test_functions.py b/ibis/backends/dask/tests/execution/test_functions.py new file mode 100644 index 000000000000..bcfa8eff54ee --- /dev/null +++ b/ibis/backends/dask/tests/execution/test_functions.py @@ -0,0 +1,222 @@ +import decimal +import functools +import math +import operator +from operator import methodcaller + +import dask.dataframe as dd +import numpy as np +import pandas as pd +import pytest +from dask.dataframe.utils import tm # noqa: E402 + +import ibis +import ibis.expr.datatypes as dt # noqa: E402 + +from ... import execute + +pytestmark = pytest.mark.dask + + +@pytest.mark.parametrize( + 'op', + [ + # comparison + operator.eq, + operator.ne, + operator.lt, + operator.le, + operator.gt, + operator.ge, + ], +) +def test_binary_operations(t, df, op): + expr = op(t.plain_float64, t.plain_int64) + result = expr.execute() + expected = op(df.plain_float64, df.plain_int64) + tm.assert_series_equal(result.compute(), expected.compute()) + + +@pytest.mark.parametrize('op', [operator.and_, operator.or_, operator.xor]) +def test_binary_boolean_operations(t, df, op): + expr = op(t.plain_int64 == 1, t.plain_int64 == 2) + result = expr.execute() + expected = op(df.plain_int64 == 1, df.plain_int64 == 2) + tm.assert_series_equal(result.compute(), expected.compute()) + + +def operate(func): + @functools.wraps(func) + def wrapper(*args, **kwargs): + try: + return func(*args, **kwargs) + except decimal.InvalidOperation: + return decimal.Decimal('NaN') + + return wrapper + + +@pytest.mark.parametrize( + ('ibis_func', 'dask_func'), + [ + (methodcaller('round'), lambda x: np.int64(round(x))), + ( + methodcaller('round', 2), + lambda x: x.quantize(decimal.Decimal('.00')), + ), + ( + methodcaller('round', 0), + lambda x: x.quantize(decimal.Decimal('0.')), + ), + (methodcaller('ceil'), lambda x: decimal.Decimal(math.ceil(x))), + (methodcaller('floor'), lambda x: decimal.Decimal(math.floor(x))), + (methodcaller('exp'), methodcaller('exp')), + ( + methodcaller('sign'), + lambda x: x if not x else decimal.Decimal(1).copy_sign(x), + ), + (methodcaller('sqrt'), operate(lambda x: x.sqrt())), + ( + methodcaller('log', 2), + operate(lambda x: x.ln() / decimal.Decimal(2).ln()), + ), + (methodcaller('ln'), operate(lambda x: x.ln())), + ( + methodcaller('log2'), + operate(lambda x: x.ln() / decimal.Decimal(2).ln()), + ), + (methodcaller('log10'), operate(lambda x: x.log10())), + ], +) +def test_math_functions_decimal(t, df, ibis_func, dask_func): + dtype = dt.Decimal(12, 3) + result = ibis_func(t.float64_as_strings.cast(dtype)).execute() + context = decimal.Context(prec=dtype.precision) + expected = df.float64_as_strings.apply( + lambda x: context.create_decimal(x).quantize( + decimal.Decimal( + '{}.{}'.format( + '0' * (dtype.precision - dtype.scale), '0' * dtype.scale + ) + ) + ), + meta=("float64_as_strings", "object"), + ).apply(dask_func, meta=("float64_as_strings", "object")) + # dask.dataframe.Series doesn't do direct item assignment + # TODO - maybe use .where instead + computed_result = result.compute() + computed_result[computed_result.apply(math.isnan)] = -99999 + computed_expected = expected.compute() + computed_expected[computed_expected.apply(math.isnan)] = -99999 + # result[result.apply(math.isnan)] = -99999 + # expected[expected.apply(math.isnan)] = -99999 + tm.assert_series_equal(computed_result, computed_expected) + + +def test_round_decimal_with_negative_places(t, df): + type = dt.Decimal(12, 3) + expr = t.float64_as_strings.cast(type).round(-1) + result = expr.execute() + expected = dd.from_pandas( + pd.Series( + list(map(decimal.Decimal, ['1.0E+2', '2.3E+2', '-1.00E+3'])), + name='float64_as_strings', + ), + npartitions=1, + ) + tm.assert_series_equal(result.compute(), expected.compute()) + + +@pytest.mark.parametrize( + ('ibis_func', 'dask_func'), + [ + (lambda x: x.clip(lower=0), lambda x: x.clip(lower=0)), + (lambda x: x.clip(lower=0.0), lambda x: x.clip(lower=0.0)), + (lambda x: x.clip(upper=0), lambda x: x.clip(upper=0)), + ( + lambda x: x.clip(lower=x - 1, upper=x + 1), + lambda x: x.clip(lower=x - 1, upper=x + 1), + ), + ( + lambda x: x.clip(lower=0, upper=1), + lambda x: x.clip(lower=0, upper=1), + ), + ( + lambda x: x.clip(lower=0, upper=1.0), + lambda x: x.clip(lower=0, upper=1.0), + ), + ], +) +def test_clip(t, df, ibis_func, dask_func): + result = ibis_func(t.float64_with_zeros).execute() + expected = dask_func(df.float64_with_zeros) + tm.assert_series_equal(result.compute(), expected.compute()) + + +@pytest.mark.parametrize( + ('ibis_func', 'dask_func'), + [ + ( + lambda x: x.quantile([0.25, 0.75]), + lambda x: list(x.quantile([0.25, 0.75])), + ) + ], +) +@pytest.mark.parametrize('column', ['float64_with_zeros', 'int64_with_zeros']) +def test_quantile_list(t, df, ibis_func, dask_func, column): + expr = ibis_func(t[column]) + result = expr.execute() + expected = dask_func(df[column]) + assert result == expected + + +@pytest.mark.parametrize( + ('ibis_func', 'dask_func'), + [ + (lambda x: x.quantile(0), lambda x: x.quantile(0)), + (lambda x: x.quantile(1), lambda x: x.quantile(1)), + (lambda x: x.quantile(0.5), lambda x: x.quantile(0.5),), + ], +) +def test_quantile_scalar(t, df, ibis_func, dask_func): + # TODO - interpolation + result = ibis_func(t.float64_with_zeros).execute() + expected = dask_func(df.float64_with_zeros) + + result = ibis_func(t.int64_with_zeros).execute() + expected = dask_func(df.int64_with_zeros) + assert result.compute() == expected.compute() + + +@pytest.mark.parametrize( + ('ibis_func', 'exc'), + [ + # no lower/upper specified + (lambda x: x.clip(), ValueError), + # out of range on quantile + (lambda x: x.quantile(5.0), ValueError), + # invalid interpolation arg + (lambda x: x.quantile(0.5, interpolation='foo'), ValueError), + ], +) +def test_arraylike_functions_transform_errors(t, df, ibis_func, exc): + with pytest.raises(exc): + ibis_func(t.float64_with_zeros).execute() + + +def test_quantile_array_access(client, t, df): + quantile = t.float64_with_zeros.quantile([0.25, 0.5]) + expr = quantile[0], quantile[1] + result = tuple(map(client.execute, expr)) + expected = tuple(df.float64_with_zeros.quantile([0.25, 0.5])) + assert result == expected + + +def test_ifelse_returning_bool(): + one = ibis.literal(1) + two = ibis.literal(2) + true = ibis.literal(True) + false = ibis.literal(False) + expr = ibis.ifelse(one + one == two, true, false) + result = execute(expr) + assert result is True diff --git a/ibis/backends/dask/tests/execution/test_join.py b/ibis/backends/dask/tests/execution/test_join.py new file mode 100644 index 000000000000..342026319f3b --- /dev/null +++ b/ibis/backends/dask/tests/execution/test_join.py @@ -0,0 +1,497 @@ +import dask.dataframe as dd +import pandas as pd +import pytest +from dask.dataframe.utils import tm +from pandas import Timedelta, date_range +from pytest import param + +import ibis +import ibis.common.exceptions as com + +from ... import connect + +# Note - computations in this file use the single threadsed scheduler (instead +# of the default multithreaded scheduler) in order to avoid a flaky interaction +# between dask and pandas in merges. There is evidence this has been fixed in +# pandas>=1.1.2 (or in other schedulers). For more background see: +# - https://github.com/dask/dask/issues/6454 +# - https://github.com/dask/dask/issues/5060 + + +pytestmark = pytest.mark.dask + + +join_type = pytest.mark.parametrize( + 'how', + [ + 'inner', + 'left', + 'right', + 'outer', + param( + 'semi', + marks=pytest.mark.xfail( + raises=NotImplementedError, reason='Semi join not implemented' + ), + ), + param( + 'anti', + marks=pytest.mark.xfail( + raises=NotImplementedError, reason='Anti join not implemented' + ), + ), + ], +) + + +@join_type +def test_join(how, left, right, df1, df2): + expr = left.join(right, left.key == right.key, how=how)[ + left, right.other_value, right.key3 + ] + result = expr.execute() + expected = dd.merge(df1, df2, how=how, on='key') + tm.assert_frame_equal( + result[expected.columns].compute(scheduler='single-threaded'), + expected.compute(scheduler='single-threaded'), + ) + + +def test_cross_join(left, right, df1, df2): + expr = left.cross_join(right)[left, right.other_value, right.key3] + result = expr.execute() + expected = dd.merge( + df1.assign(dummy=1), df2.assign(dummy=1), how='inner', on='dummy' + ).rename(columns=dict(key_x='key')) + del expected['dummy'], expected['key_y'] + tm.assert_frame_equal( + result[expected.columns].compute(scheduler='single-threaded'), + expected.compute(scheduler='single-threaded'), + ) + + +@join_type +def test_join_project_left_table(how, left, right, df1, df2): + expr = left.join(right, left.key == right.key, how=how)[left, right.key3] + result = expr.execute() + expected = dd.merge(df1, df2, how=how, on='key')[ + list(left.columns) + ['key3'] + ] + tm.assert_frame_equal( + result[expected.columns].compute(scheduler='single-threaded'), + expected.compute(scheduler='single-threaded'), + ) + + +def test_cross_join_project_left_table(left, right, df1, df2): + expr = left.cross_join(right)[left, right.key3] + result = expr.execute() + expected = dd.merge( + df1.assign(dummy=1), df2.assign(dummy=1), how='inner', on='dummy' + ).rename(columns=dict(key_x='key'))[list(left.columns) + ['key3']] + tm.assert_frame_equal( + result[expected.columns].compute(scheduler='single-threaded'), + expected.compute(scheduler='single-threaded'), + ) + + +@join_type +def test_join_with_multiple_predicates(how, left, right, df1, df2): + expr = left.join( + right, [left.key == right.key, left.key2 == right.key3], how=how + )[left, right.key3, right.other_value] + result = expr.execute() + expected = dd.merge( + df1, df2, how=how, left_on=['key', 'key2'], right_on=['key', 'key3'] + ).reset_index(drop=True) + tm.assert_frame_equal( + result[expected.columns].compute(scheduler='single-threaded'), + expected.compute(scheduler='single-threaded'), + ) + + +@join_type +def test_join_with_multiple_predicates_written_as_one( + how, left, right, df1, df2 +): + predicate = (left.key == right.key) & (left.key2 == right.key3) + expr = left.join(right, predicate, how=how)[ + left, right.key3, right.other_value + ] + result = expr.execute() + expected = dd.merge( + df1, df2, how=how, left_on=['key', 'key2'], right_on=['key', 'key3'] + ).reset_index(drop=True) + tm.assert_frame_equal( + result[expected.columns].compute(scheduler='single-threaded'), + expected.compute(scheduler='single-threaded'), + ) + + +@join_type +def test_join_with_invalid_predicates(how, left, right): + predicate = (left.key == right.key) & (left.key2 <= right.key3) + expr = left.join(right, predicate, how=how) + with pytest.raises(TypeError): + expr.execute() + + predicate = left.key >= right.key + expr = left.join(right, predicate, how=how) + with pytest.raises(TypeError): + expr.execute() + + +@join_type +@pytest.mark.xfail(reason='Hard to detect this case') +def test_join_with_duplicate_non_key_columns(how, left, right, df1, df2): + left = left.mutate(x=left.value * 2) + right = right.mutate(x=right.other_value * 3) + expr = left.join(right, left.key == right.key, how=how) + + # This is undefined behavior because `x` is duplicated. This is difficult + # to detect + with pytest.raises(ValueError): + expr.execute() + + +@join_type +def test_join_with_duplicate_non_key_columns_not_selected( + how, left, right, df1, df2 +): + left = left.mutate(x=left.value * 2) + right = right.mutate(x=right.other_value * 3) + right = right[['key', 'other_value']] + expr = left.join(right, left.key == right.key, how=how)[ + left, right.other_value + ] + result = expr.execute() + expected = dd.merge( + df1.assign(x=df1.value * 2), + df2[['key', 'other_value']], + how=how, + on='key', + ) + tm.assert_frame_equal( + result[expected.columns].compute(scheduler='single-threaded'), + expected.compute(scheduler='single-threaded'), + ) + + +@join_type +def test_join_with_post_expression_selection(how, left, right, df1, df2): + join = left.join(right, left.key == right.key, how=how) + expr = join[left.key, left.value, right.other_value] + result = expr.execute() + expected = dd.merge(df1, df2, on='key', how=how)[ + ['key', 'value', 'other_value'] + ] + tm.assert_frame_equal( + result[expected.columns].compute(scheduler='single-threaded'), + expected.compute(scheduler='single-threaded'), + ) + + +@join_type +def test_join_with_post_expression_filter(how, left): + lhs = left[['key', 'key2']] + rhs = left[['key2', 'value']] + + joined = lhs.join(rhs, 'key2', how=how) + projected = joined[lhs, rhs.value] + expr = projected[projected.value == 4] + result = expr.execute() + + df1 = lhs.execute() + df2 = rhs.execute() + expected = dd.merge(df1, df2, on='key2', how=how) + expected = expected.loc[expected.value == 4].reset_index(drop=True) + + tm.assert_frame_equal( + result.compute(scheduler='single-threaded'), + expected.compute(scheduler='single-threaded'), + ) + + +@join_type +def test_multi_join_with_post_expression_filter(how, left, df1): + lhs = left[['key', 'key2']] + rhs = left[['key2', 'value']] + rhs2 = left[['key2', 'value']].relabel(dict(value='value2')) + + joined = lhs.join(rhs, 'key2', how=how) + projected = joined[lhs, rhs.value] + filtered = projected[projected.value == 4] + + joined2 = filtered.join(rhs2, 'key2') + projected2 = joined2[filtered.key, rhs2.value2] + expr = projected2[projected2.value2 == 3] + + result = expr.execute() + + df1 = lhs.execute() + df2 = rhs.execute() + df3 = rhs2.execute() + expected = dd.merge(df1, df2, on='key2', how=how) + expected = expected.loc[expected.value == 4].reset_index(drop=True) + expected = dd.merge(expected, df3, on='key2')[['key', 'value2']] + expected = expected.loc[expected.value2 == 3].reset_index(drop=True) + + tm.assert_frame_equal( + result.compute(scheduler='single-threaded'), + expected.compute(scheduler='single-threaded'), + ) + + +@pytest.mark.xfail(reason="TODO - execute_materialized_join - #2553") +@join_type +def test_join_with_non_trivial_key(how, left, right, df1, df2): + # also test that the order of operands in the predicate doesn't matter + join = left.join(right, right.key.length() == left.key.length(), how=how) + expr = join[left.key, left.value, right.other_value] + result = expr.execute() + + expected = ( + dd.merge( + df1.assign(key_len=df1.key.str.len()), + df2.assign(key_len=df2.key.str.len()), + on='key_len', + how=how, + ) + .drop(['key_len', 'key_y', 'key2', 'key3'], axis=1) + .rename(columns={'key_x': 'key'}) + ) + tm.assert_frame_equal( + result[expected.columns].compute(scheduler='single-threaded'), + expected.compute(scheduler='single-threaded'), + ) + + +@pytest.mark.xfail(reason="TODO - execute_materialized_join - #2553") +@join_type +def test_join_with_non_trivial_key_project_table(how, left, right, df1, df2): + # also test that the order of operands in the predicate doesn't matter + join = left.join(right, right.key.length() == left.key.length(), how=how) + expr = join[left, right.other_value] + expr = expr[expr.key.length() == 1] + result = expr.execute() + + expected = ( + dd.merge( + df1.assign(key_len=df1.key.str.len()), + df2.assign(key_len=df2.key.str.len()), + on='key_len', + how=how, + ) + .drop(['key_len', 'key_y', 'key2', 'key3'], axis=1) + .rename(columns={'key_x': 'key'}) + ) + expected = expected.loc[expected.key.str.len() == 1] + tm.assert_frame_equal( + result[expected.columns].compute(scheduler='single-threaded'), + expected.compute(scheduler='single-threaded'), + ) + + +@join_type +def test_join_with_project_right_duplicate_column(client, how, left, df1, df3): + # also test that the order of operands in the predicate doesn't matter + right = client.table('df3') + join = left.join(right, ['key'], how=how) + expr = join[left.key, right.key2, right.other_value] + result = expr.execute() + + expected = ( + dd.merge(df1, df3, on='key', how=how) + .drop(['key2_x', 'key3', 'value'], axis=1) + .rename(columns={'key2_y': 'key2'}) + ) + tm.assert_frame_equal( + result[expected.columns].compute(scheduler='single-threaded'), + expected.compute(scheduler='single-threaded'), + ) + + +def test_join_with_window_function( + players_base, players_df, batting, batting_df +): + players = players_base + + # this should be semi_join + tbl = batting.left_join(players, ['playerID']) + t = tbl[batting.G, batting.playerID, batting.teamID] + expr = t.groupby(t.teamID).mutate( + team_avg=lambda d: d.G.mean(), + demeaned_by_player=lambda d: d.G - d.G.mean(), + ) + result = expr.execute() + + expected = dd.merge( + batting_df, players_df[['playerID']], on='playerID', how='left' + )[['G', 'playerID', 'teamID']] + team_avg = expected.groupby('teamID').G.transform('mean') + expected = expected.assign( + team_avg=team_avg, demeaned_by_player=lambda df: df.G - team_avg + ) + + tm.assert_frame_equal( + result[expected.columns].compute(scheduler='single-threaded'), + expected.compute(scheduler='single-threaded'), + ) + + +merge_asof_minversion = pytest.mark.skipif( + pd.__version__ < '0.19.2', + reason="at least pandas-0.19.2 required for merge_asof", +) + + +@merge_asof_minversion +def test_asof_join(time_left, time_right, time_df1, time_df2): + expr = time_left.asof_join(time_right, 'time')[ + time_left, time_right.other_value + ] + result = expr.execute() + expected = dd.merge_asof(time_df1, time_df2, on='time') + tm.assert_frame_equal( + result[expected.columns].compute(scheduler='single-threaded'), + expected.compute(scheduler='single-threaded'), + ) + + +@merge_asof_minversion +def test_asof_join_predicate(time_left, time_right, time_df1, time_df2): + expr = time_left.asof_join(time_right, time_left.time == time_right.time)[ + time_left, time_right.other_value + ] + result = expr.execute() + expected = dd.merge_asof(time_df1, time_df2, on='time') + tm.assert_frame_equal( + result[expected.columns].compute(scheduler='single-threaded'), + expected.compute(scheduler='single-threaded'), + ) + + +@merge_asof_minversion +def test_keyed_asof_join( + time_keyed_left, time_keyed_right, time_keyed_df1, time_keyed_df2 +): + expr = time_keyed_left.asof_join(time_keyed_right, 'time', by='key')[ + time_keyed_left, time_keyed_right.other_value + ] + result = expr.execute() + expected = dd.merge_asof( + time_keyed_df1, time_keyed_df2, on='time', by='key' + ) + tm.assert_frame_equal( + result[expected.columns].compute(scheduler='single-threaded'), + expected.compute(scheduler='single-threaded'), + ) + + +@merge_asof_minversion +def test_keyed_asof_join_with_tolerance( + time_keyed_left, time_keyed_right, time_keyed_df1, time_keyed_df2 +): + expr = time_keyed_left.asof_join( + time_keyed_right, 'time', by='key', tolerance=2 * ibis.interval(days=1) + )[time_keyed_left, time_keyed_right.other_value] + result = expr.execute() + expected = dd.merge_asof( + time_keyed_df1, + time_keyed_df2, + on='time', + by='key', + tolerance=Timedelta('2D'), + ) + tm.assert_frame_equal( + result[expected.columns].compute(scheduler='single-threaded'), + expected.compute(scheduler='single-threaded'), + ) + + +@pytest.mark.parametrize( + "how", + [ + "left", + pytest.param( + "right", + marks=pytest.mark.xfail( + raises=AttributeError, reason="right_join is not an ibis API" + ), + ), + "inner", + "outer", + ], +) +@pytest.mark.parametrize( + "func", + [ + pytest.param(lambda join: join["a0", "a1"], id="tuple"), + pytest.param(lambda join: join[["a0", "a1"]], id="list"), + pytest.param(lambda join: join.select(["a0", "a1"]), id="select"), + ], +) +@pytest.mark.xfail( + raises=(com.IbisError, AttributeError), + reason="Select from unambiguous joins not implemented", +) +def test_select_on_unambiguous_join(how, func): + df_t = dd.from_pandas( + pd.DataFrame(dict(a0=[1, 2, 3], b1=list("aab"))), npartitions=1, + ) + df_s = dd.from_pandas( + pd.DataFrame(dict(a1=[2, 3, 4], b2=list("abc"))), npartitions=1, + ) + con = connect({"t": df_t, "s": df_s}) + t = con.table("t") + s = con.table("s") + method = getattr(t, "{}_join".format(how)) + join = method(s, t.b1 == s.b2) + expected = dd.merge(df_t, df_s, left_on=["b1"], right_on=["b2"], how=how)[ + ["a0", "a1"] + ] + assert not expected.compute(scheduler='single-threaded').empty + expr = func(join) + result = expr.execute() + tm.assert_frame_equal( + result.compute(scheduler='single-threaded'), + expected.compute(scheduler='single-threaded'), + ) + + +@pytest.mark.parametrize( + "func", + [ + pytest.param(lambda join: join["a0", "a1"], id="tuple"), + pytest.param(lambda join: join[["a0", "a1"]], id="list"), + pytest.param(lambda join: join.select(["a0", "a1"]), id="select"), + ], +) +@pytest.mark.xfail( + raises=(com.IbisError, AttributeError), + reason="Select from unambiguous joins not implemented", +) +@merge_asof_minversion +def test_select_on_unambiguous_asof_join(func): + df_t = dd.from_pandas( + pd.DataFrame(dict(a0=[1, 2, 3], b1=date_range("20180101", periods=3))), + npartitions=1, + ) + df_s = dd.from_pandas( + pd.DataFrame(dict(a1=[2, 3, 4], b2=date_range("20171230", periods=3))), + npartitions=1, + ) + con = connect({"t": df_t, "s": df_s}) + t = con.table("t") + s = con.table("s") + join = t.asof_join(s, t.b1 == s.b2) + expected = dd.merge_asof(df_t, df_s, left_on=["b1"], right_on=["b2"])[ + ["a0", "a1"] + ] + assert not expected.compute(scheduler='single-threaded').empty + expr = func(join) + result = expr.execute() + tm.assert_frame_equal( + result.compute(scheduler='single-threaded'), + expected.compute(scheduler='single-threaded'), + ) diff --git a/ibis/backends/dask/tests/execution/test_maps.py b/ibis/backends/dask/tests/execution/test_maps.py new file mode 100644 index 000000000000..d5ffdb0bb52d --- /dev/null +++ b/ibis/backends/dask/tests/execution/test_maps.py @@ -0,0 +1,98 @@ +import dask.dataframe as dd +import pandas as pd +from dask.dataframe.utils import tm + +import ibis + + +def test_map_length_expr(t): + expr = t.map_of_integers_strings.length() + result = expr.execute() + expected = dd.from_pandas( + pd.Series([0, None, 2], name='map_of_integers_strings'), npartitions=1, + ) + tm.assert_series_equal(result.compute(), expected.compute()) + + +def test_map_value_for_key_expr(t): + expr = t.map_of_integers_strings[1] + result = expr.execute() + expected = dd.from_pandas( + pd.Series([None, None, 'a'], name='map_of_integers_strings'), + npartitions=1, + ) + tm.assert_series_equal(result.compute(), expected.compute()) + + +def test_map_value_or_default_for_key_expr(t): + expr = t.map_of_complex_values.get('a') + result = expr.execute() + expected = dd.from_pandas( + pd.Series( + [None, [1, 2, 3], None], + dtype='object', + name='map_of_complex_values', + ), + npartitions=1, + ) + tm.assert_series_equal(result.compute(), expected.compute()) + + +def safe_sorter(element): + return sorted(element) if isinstance(element, list) else element + + +def test_map_keys_expr(t): + expr = t.map_of_strings_integers.keys() + result = expr.execute().map(safe_sorter) + expected = dd.from_pandas( + pd.Series( + [['a', 'b'], None, []], + dtype='object', + name='map_of_strings_integers', + ), + npartitions=1, + ) + tm.assert_series_equal(result.compute(), expected.compute()) + + +def test_map_values_expr(t): + expr = t.map_of_complex_values.values() + result = expr.execute().map(safe_sorter) + expected = dd.from_pandas( + pd.Series( + [None, [[], [1, 2, 3]], []], + dtype='object', + name='map_of_complex_values', + ), + npartitions=1, + ) + tm.assert_series_equal(result.compute(), expected.compute()) + + +def test_map_concat_expr(t): + expr = t.map_of_complex_values + {'b': [4, 5, 6], 'c': [], 'a': []} + result = expr.execute() + expected = dd.from_pandas( + pd.Series( + [ + None, + {'a': [], 'b': [4, 5, 6], 'c': []}, + {'b': [4, 5, 6], 'c': [], 'a': []}, + ], + dtype='object', + name='map_of_complex_values', + ), + npartitions=1, + ) + tm.assert_series_equal(result.compute(), expected.compute()) + + +def test_map_value_for_key_literal_broadcast(t): + lookup_table = ibis.literal({'a': 1, 'b': 2, 'c': 3, 'd': 4}) + expr = lookup_table.get(t.dup_strings) + result = expr.execute() + expected = dd.from_pandas( + pd.Series([4, 1, 4], name='dup_strings'), npartitions=1, + ) + tm.assert_series_equal(result.compute(), expected.compute()) diff --git a/ibis/backends/dask/tests/execution/test_operations.py b/ibis/backends/dask/tests/execution/test_operations.py new file mode 100644 index 000000000000..04de99c70f36 --- /dev/null +++ b/ibis/backends/dask/tests/execution/test_operations.py @@ -0,0 +1,927 @@ +import operator +from operator import methodcaller + +import dask.array as da +import dask.dataframe as dd +import numpy as np +import numpy.testing as npt +import pandas as pd +import pytest +from dask.dataframe.utils import tm + +import ibis +import ibis.expr.datatypes as dt + +from ... import connect, execute + +pytestmark = pytest.mark.dask + + +def test_table_column(t, df): + expr = t.plain_int64 + result = expr.execute() + expected = df.plain_int64 + tm.assert_series_equal(result.compute(), expected.compute()) + + +def test_literal(client): + assert client.execute(ibis.literal(1)) == 1 + + +def test_read_with_undiscoverable_type(client): + with pytest.raises(TypeError): + client.table('df') + + +def test_selection(t, df): + expr = t[ + ((t.plain_strings == 'a') | (t.plain_int64 == 3)) + & (t.dup_strings == 'd') + ] + result = expr.execute() + expected = df[ + ((df.plain_strings == 'a') | (df.plain_int64 == 3)) + & (df.dup_strings == 'd') + ].reset_index(drop=True) + tm.assert_frame_equal( + result[expected.columns].compute(), expected.compute() + ) + + +def test_mutate(t, df): + expr = t.mutate(x=t.plain_int64 + 1, y=t.plain_int64 * 2) + result = expr.execute() + expected = df.assign(x=df.plain_int64 + 1, y=df.plain_int64 * 2) + tm.assert_frame_equal( + result[expected.columns].compute(), expected.compute() + ) + + +@pytest.mark.xfail(reason='TODO - windowing - #2553') +def test_project_scope_does_not_override(t, df): + col = t.plain_int64 + expr = t[ + [ + col.name('new_col'), + col.sum() + .over(ibis.window(group_by='dup_strings')) + .name('grouped'), + ] + ] + result = expr.execute() + expected = dd.concat( + [ + df[['plain_int64', 'dup_strings']].rename( + columns={'plain_int64': 'new_col'} + ), + df.groupby('dup_strings') + .plain_int64.transform('sum') + .reset_index(drop=True) + .rename('grouped'), + ], + axis=1, + )[['new_col', 'grouped']] + tm.assert_frame_equal(result.compute(), expected.compute()) + + +@pytest.mark.xfail(reason="TODO - aggregations - #2553") +@pytest.mark.parametrize( + 'where', + [ + lambda t: None, + lambda t: t.dup_strings == 'd', + lambda t: (t.dup_strings == 'd') | (t.plain_int64 < 100), + ], +) +@pytest.mark.parametrize( + ('ibis_func', 'dask_func'), + [ + (methodcaller('abs'), np.abs), + (methodcaller('ceil'), np.ceil), + (methodcaller('exp'), np.exp), + (methodcaller('floor'), np.floor), + (methodcaller('ln'), np.log), + (methodcaller('log10'), np.log10), + (methodcaller('log', 2), lambda x: np.log(x) / np.log(2)), + (methodcaller('log2'), np.log2), + (methodcaller('round', 0), lambda x: x.round(0).astype('int64')), + (methodcaller('round', -2), methodcaller('round', -2)), + (methodcaller('round', 2), methodcaller('round', 2)), + (methodcaller('round'), lambda x: x.round().astype('int64')), + (methodcaller('sign'), np.sign), + (methodcaller('sqrt'), np.sqrt), + ], +) +def test_aggregation_group_by(t, df, where, ibis_func, dask_func): + ibis_where = where(t) + expr = t.group_by(t.dup_strings).aggregate( + avg_plain_int64=t.plain_int64.mean(where=ibis_where), + sum_plain_float64=t.plain_float64.sum(where=ibis_where), + mean_float64_positive=ibis_func(t.float64_positive).mean( + where=ibis_where + ), + neg_mean_int64_with_zeros=(-t.int64_with_zeros).mean(where=ibis_where), + nunique_dup_ints=t.dup_ints.nunique(), + ) + result = expr.execute() + + dask_where = where(df) + mask = slice(None) if dask_where is None else dask_where + expected = ( + df.groupby('dup_strings') + .agg( + { + 'plain_int64': lambda x, mask=mask: x[mask].mean(), + 'plain_float64': lambda x, mask=mask: x[mask].sum(), + 'dup_ints': 'nunique', + 'float64_positive': ( + lambda x, mask=mask, func=dask_func: func(x[mask]).mean() + ), + 'int64_with_zeros': lambda x, mask=mask: (-x[mask]).mean(), + } + ) + .reset_index() + .rename( + columns={ + 'plain_int64': 'avg_plain_int64', + 'plain_float64': 'sum_plain_float64', + 'dup_ints': 'nunique_dup_ints', + 'float64_positive': 'mean_float64_positive', + 'int64_with_zeros': 'neg_mean_int64_with_zeros', + } + ) + ) + # TODO(phillipc): Why does pandas not return floating point values here? + expected['avg_plain_int64'] = expected.avg_plain_int64.astype('float64') + result['avg_plain_int64'] = result.avg_plain_int64.astype('float64') + expected[ + 'neg_mean_int64_with_zeros' + ] = expected.neg_mean_int64_with_zeros.astype('float64') + result[ + 'neg_mean_int64_with_zeros' + ] = result.neg_mean_int64_with_zeros.astype('float64') + expected['mean_float64_positive'] = expected.mean_float64_positive.astype( + 'float64' + ) + result['mean_float64_positive'] = result.mean_float64_positive.astype( + 'float64' + ) + lhs = result[expected.columns] + rhs = expected + tm.assert_frame_equal(lhs.compute(), rhs.compute()) + + +@pytest.mark.xfail(reason="TODO - aggregations - #2553") +def test_aggregation_without_group_by(t, df): + expr = t.aggregate( + avg_plain_int64=t.plain_int64.mean(), + sum_plain_float64=t.plain_float64.sum(), + ) + result = expr.execute()[['avg_plain_int64', 'sum_plain_float64']] + new_names = { + 'plain_float64': 'sum_plain_float64', + 'plain_int64': 'avg_plain_int64', + } + expected = ( + dd.from_array( + [df['plain_int64'].mean(), df['plain_float64'].sum()], + index=['plain_int64', 'plain_float64'], + ) + .to_frame() + .T.rename(columns=new_names) + ) + tm.assert_frame_equal( + result[expected.columns].compute(), expected.compute() + ) + + +def test_group_by_with_having(t, df): + expr = ( + t.group_by(t.dup_strings) + .having(t.plain_float64.sum() == 5) + .aggregate(avg_a=t.plain_int64.mean(), sum_c=t.plain_float64.sum()) + ) + result = expr.execute() + + expected = ( + df.groupby('dup_strings') + .agg({'plain_int64': 'mean', 'plain_float64': 'sum'}) + .reset_index() + .rename(columns={'plain_int64': 'avg_a', 'plain_float64': 'sum_c'}) + ) + expected = expected.loc[expected.sum_c == 5, ['avg_a', 'sum_c']] + + tm.assert_frame_equal( + result[expected.columns].compute(), expected.compute() + ) + + +def test_group_by_rename_key(t, df): + expr = t.groupby(t.dup_strings.name('foo')).aggregate( + dup_string_count=t.dup_strings.count() + ) + + assert 'foo' in expr.schema() + result = expr.execute() + assert 'foo' in result.columns + + expected = ( + df.groupby('dup_strings') + .dup_strings.count() + .rename('dup_string_count') + .reset_index() + .rename(columns={'dup_strings': 'foo'}) + ) + tm.assert_frame_equal(result.compute(), expected.compute()) + + +@pytest.mark.parametrize('reduction', ['mean', 'sum', 'count', 'std', 'var']) +@pytest.mark.parametrize( + 'where', + [ + lambda t: (t.plain_strings == 'a') | (t.plain_strings == 'c'), + lambda t: (t.dup_strings == 'd') + & ((t.plain_int64 == 1) | (t.plain_int64 == 3)), + lambda t: None, + ], +) +def test_reduction(t, df, reduction, where): + func = getattr(t.plain_int64, reduction) + mask = where(t) + expr = func(where=mask) + result = expr.execute() + + df_mask = where(df) + expected_func = getattr( + df.loc[df_mask if df_mask is not None else slice(None), 'plain_int64'], + reduction, + ) + expected = expected_func() + assert result.compute() == expected.compute() + + +@pytest.mark.xfail(NotImplementedError, reason="TODO - aggregations - #2553") +@pytest.mark.parametrize( + 'reduction', + [ + lambda x: x.any(), + lambda x: x.all(), + lambda x: ~(x.any()), + lambda x: ~(x.all()), + ], +) +def test_boolean_aggregation(t, df, reduction): + expr = reduction(t.plain_int64 == 1) + result = expr.execute() + expected = reduction(df.plain_int64 == 1) + assert result.compute() == expected.compute() + + +@pytest.mark.parametrize('column', ['float64_with_zeros', 'int64_with_zeros']) +def test_null_if_zero(t, df, column): + expr = t[column].nullifzero() + result = expr.execute() + expected = df[column].replace(0, np.nan) + tm.assert_series_equal(result.compute(), expected.compute()) + + +@pytest.mark.parametrize( + ('left', 'right', 'expected', 'compare'), + [ + pytest.param( + lambda t: ibis.literal(1), + lambda t: ibis.literal(1), + lambda df: np.nan, + np.testing.assert_array_equal, # treats NaNs as equal + id='literal_literal_equal', + ), + pytest.param( + lambda t: ibis.literal(1), + lambda t: ibis.literal(2), + lambda df: 1, + np.testing.assert_equal, + id='literal_literal_not_equal', + ), + pytest.param( + lambda t: t.dup_strings, + lambda t: ibis.literal('a'), + lambda df: df.dup_strings.where(df.dup_strings != 'a'), + tm.assert_series_equal, + id='series_literal', + ), + pytest.param( + lambda t: t.dup_strings, + lambda t: t.dup_strings, + lambda df: df.dup_strings.where(df.dup_strings != df.dup_strings), + tm.assert_series_equal, + id='series_series', + ), + pytest.param( + lambda t: ibis.literal('a'), + lambda t: t.dup_strings, + lambda df: dd.from_array( + da.where(df.dup_strings.eq('a').values, np.nan, 'a') + ), + tm.assert_series_equal, + id='literal_series', + ), + ], +) +def test_nullif(t, df, left, right, expected, compare): + expr = left(t).nullif(right(t)) + result = execute(expr) + if isinstance(result, (dd.Series, dd.DataFrame)): + compare(result.compute(), expected(df).compute()) + else: + compare(result, expected(df)) + + +def test_nullif_inf(): + df = dd.from_pandas( + pd.DataFrame({'a': [np.inf, 3.14, -np.inf, 42.0]}), npartitions=1, + ) + con = connect(dict(t=df)) + t = con.table('t') + expr = t.a.nullif(np.inf).nullif(-np.inf) + result = expr.execute() + expected = dd.from_pandas( + pd.Series([np.nan, 3.14, np.nan, 42.0], name='a'), npartitions=1, + ) + tm.assert_series_equal(result.compute(), expected.compute()) + + +def test_group_concat(t, df): + expr = t.groupby(t.dup_strings).aggregate( + foo=t.plain_int64.group_concat(',') + ) + result = expr.execute() + expected = ( + df.groupby('dup_strings') + .apply(lambda df: ','.join(df.plain_int64.astype(str))) + .reset_index() + .rename(columns={0: 'foo'}) + ) + tm.assert_frame_equal( + result[expected.columns].compute(), expected.compute() + ) + + +@pytest.mark.parametrize('offset', [0, 2]) +def test_frame_limit(t, df, offset): + n = 5 + df_expr = t.limit(n, offset=offset) + result = df_expr.execute() + expected = df.loc[offset : offset + n].reset_index(drop=True) + tm.assert_frame_equal( + result[expected.columns].compute(), expected.compute() + ) + + +@pytest.mark.xfail( + raises=AttributeError, reason='TableColumn does not implement limit' +) +@pytest.mark.parametrize('offset', [0, 2]) +def test_series_limit(t, df, offset): + n = 5 + s_expr = t.plain_int64.limit(n, offset=offset) + result = s_expr.execute() + tm.assert_series_equal(result, df.plain_int64.iloc[offset : offset + n]) + + +@pytest.mark.xfail(reason="TODO - sorting - #2553") +@pytest.mark.parametrize( + ('key', 'dask_by', 'dask_ascending'), + [ + (lambda t, col: [ibis.desc(t[col])], lambda col: [col], False), + ( + lambda t, col: [t[col], ibis.desc(t.plain_int64)], + lambda col: [col, 'plain_int64'], + [True, False], + ), + ( + lambda t, col: [ibis.desc(t.plain_int64 * 2)], + lambda col: ['plain_int64'], + False, + ), + ], +) +@pytest.mark.parametrize( + 'column', + ['plain_datetimes_naive', 'plain_datetimes_ny', 'plain_datetimes_utc'], +) +def test_sort_by(t, df, column, key, dask_by, dask_ascending): + expr = t.sort_by(key(t, column)) + result = expr.execute() + expected = ( + df.compute() + .sort_values(dask_by(column), ascending=dask_ascending) + .reset_index(drop=True) + ) + tm.assert_frame_equal(result[expected.columns].compute(), expected) + + +@pytest.mark.xfail(reason="TODO - sorting - #2553") +def test_complex_sort_by(t, df): + expr = t.sort_by( + [ibis.desc(t.plain_int64 * t.plain_float64), t.plain_float64] + ) + result = expr.execute() + expected = ( + df.assign(foo=df.plain_int64 * df.plain_float64) + .sort_values(['foo', 'plain_float64'], ascending=[False, True]) + .drop(['foo'], axis=1) + .reset_index(drop=True) + ) + + tm.assert_frame_equal( + result[expected.columns].compute(), expected.compute() + ) + + +def test_distinct(t, df): + expr = t.dup_strings.distinct() + result = expr.execute() + expected = df.dup_strings.unique() + tm.assert_series_equal(result.compute(), expected.compute()) + + +def test_count_distinct(t, df): + expr = t.dup_strings.nunique() + result = expr.execute() + expected = df.dup_strings.nunique() + assert result.compute() == expected.compute() + + +def test_value_counts(t, df): + expr = t.dup_strings.value_counts() + result = expr.execute() + expected = ( + df.compute() + .dup_strings.value_counts() + .reset_index() + .rename(columns={'dup_strings': 'count'}) + .rename(columns={'index': 'dup_strings'}) + .sort_values(['dup_strings']) + .reset_index(drop=True) + ) + tm.assert_frame_equal(result[expected.columns].compute(), expected) + + +def test_table_count(t, df): + expr = t.count() + result = expr.execute() + expected = len(df) + assert result == expected + + +@pytest.mark.xfail(reason="TODO - grouping - #2553") +def test_weighted_average(t, df): + expr = t.groupby(t.dup_strings).aggregate( + avg=(t.plain_float64 * t.plain_int64).sum() / t.plain_int64.sum() + ) + result = expr.execute() + expected = ( + df.groupby('dup_strings') + .apply( + lambda df: (df.plain_int64 * df.plain_float64).sum() + / df.plain_int64.sum() + ) + .reset_index() + .rename(columns={0: 'avg'}) + ) + tm.assert_frame_equal( + result[expected.columns].compute(), expected.compute() + ) + + +def test_group_by_multiple_keys(t, df): + expr = t.groupby([t.dup_strings, t.dup_ints]).aggregate( + avg_plain_float64=t.plain_float64.mean() + ) + result = expr.execute() + expected = ( + df.groupby(['dup_strings', 'dup_ints']) + .agg({'plain_float64': 'mean'}) + .reset_index() + .rename(columns={'plain_float64': 'avg_plain_float64'}) + ) + tm.assert_frame_equal( + result[expected.columns].compute(), expected.compute() + ) + + +def test_mutate_after_group_by(t, df): + gb = t.groupby(t.dup_strings).aggregate( + avg_plain_float64=t.plain_float64.mean() + ) + expr = gb.mutate(x=gb.avg_plain_float64) + result = expr.execute() + expected = ( + df.groupby('dup_strings') + .agg({'plain_float64': 'mean'}) + .reset_index() + .rename(columns={'plain_float64': 'avg_plain_float64'}) + ) + expected = expected.assign(x=expected.avg_plain_float64) + tm.assert_frame_equal( + result[expected.columns].compute(), expected.compute() + ) + + +@pytest.mark.xfail(reason="TODO - grouping - #2553") +def test_groupby_with_unnamed_arithmetic(t, df): + expr = t.groupby(t.dup_strings).aggregate( + naive_variance=( + (t.plain_float64 ** 2).sum() - t.plain_float64.mean() ** 2 + ) + / t.plain_float64.count() + ) + result = expr.execute() + expected = ( + df.compute() + .groupby('dup_strings') + .agg( + { + 'plain_float64': lambda x: ((x ** 2).sum() - x.mean() ** 2) + / x.count() + } + ) + .reset_index() + .rename(columns={'plain_float64': 'naive_variance'}) + ) + tm.assert_frame_equal(result[expected.columns].compute(), expected) + + +def test_isnull(t, df): + expr = t.strings_with_nulls.isnull() + result = expr.execute() + expected = df.strings_with_nulls.isnull() + tm.assert_series_equal(result.compute(), expected.compute()) + + +def test_notnull(t, df): + expr = t.strings_with_nulls.notnull() + result = expr.execute() + expected = df.strings_with_nulls.notnull() + tm.assert_series_equal(result.compute(), expected.compute()) + + +@pytest.mark.parametrize('raw_value', [0.0, 1.0]) +def test_scalar_parameter(t, df, raw_value): + value = ibis.param(dt.double) + expr = t.float64_with_zeros == value + result = expr.execute(params={value: raw_value}) + expected = df.float64_with_zeros == raw_value + tm.assert_series_equal(result.compute(), expected.compute()) + + +@pytest.mark.parametrize('elements', [[1], (1,), {1}, frozenset({1})]) +def test_isin(t, df, elements): + expr = t.plain_float64.isin(elements) + expected = df.plain_float64.isin(elements) + result = expr.execute() + tm.assert_series_equal(result.compute(), expected.compute()) + + +@pytest.mark.parametrize('elements', [[1], (1,), {1}, frozenset({1})]) +def test_notin(t, df, elements): + expr = t.plain_float64.notin(elements) + expected = ~df.plain_float64.isin(elements) + result = expr.execute() + tm.assert_series_equal(result.compute(), expected.compute()) + + +@pytest.mark.xfail(reason="TODO - grouping - #2553") +def test_cast_on_group_by(t, df): + expr = t.groupby(t.dup_strings).aggregate( + casted=(t.float64_with_zeros == 0).cast('int64').sum() + ) + result = expr.execute() + expected = ( + df.compute() + .groupby('dup_strings') + .float64_with_zeros.apply(lambda s: (s == 0).astype('int64').sum()) + .reset_index() + .rename(columns={'float64_with_zeros': 'casted'}) + ) + tm.assert_frame_equal(result.compute(), expected) + + +@pytest.mark.parametrize( + 'op', + [ + operator.add, + operator.mul, + operator.sub, + operator.truediv, + operator.floordiv, + operator.mod, + operator.pow, + ], + ids=operator.attrgetter('__name__'), +) +@pytest.mark.parametrize('args', [lambda c: (1.0, c), lambda c: (c, 1.0)]) +def test_left_binary_op(t, df, op, args): + expr = op(*args(t.float64_with_zeros)) + result = expr.execute() + expected = op(*args(df.float64_with_zeros)) + tm.assert_series_equal(result.compute(), expected.compute()) + + +@pytest.mark.xfail(reason="TODO - aggregations - #2553") +@pytest.mark.parametrize( + 'op', + [ + operator.add, + operator.mul, + operator.sub, + operator.truediv, + operator.floordiv, + operator.mod, + operator.pow, + ], + ids=operator.attrgetter('__name__'), +) +@pytest.mark.parametrize('argfunc', [lambda c: (1.0, c), lambda c: (c, 1.0)]) +def test_left_binary_op_gb(t, df, op, argfunc): + expr = t.groupby('dup_strings').aggregate( + foo=op(*argfunc(t.float64_with_zeros)).sum() + ) + result = expr.execute() + expected = ( + df.groupby('dup_strings') + .float64_with_zeros.apply(lambda s: op(*argfunc(s)).sum()) + .reset_index() + .rename(columns={'float64_with_zeros': 'foo'}) + ) + tm.assert_frame_equal(result.compute(), expected.compute()) + + +def test_where_series(t, df): + col_expr = t['plain_int64'] + result = ibis.where(col_expr > col_expr.mean(), col_expr, 0.0).execute() + + ser = df['plain_int64'] + expected = ser.where(ser > ser.mean(), other=0.0) + + tm.assert_series_equal(result.compute(), expected.compute()) + + +@pytest.mark.parametrize( + ('cond', 'expected_func'), + [ + (True, lambda df: df['plain_int64']), + (False, lambda df: dd.from_array(np.repeat(3.0, len(df)))), + ], +) +def test_where_scalar(t, df, cond, expected_func): + expr = ibis.where(cond, t['plain_int64'], 3.0) + result = expr.execute() + expected = expected_func(df) + tm.assert_series_equal(result.compute(), expected.compute()) + + +def test_where_long(batting, batting_df): + col_expr = batting['AB'] + result = ibis.where(col_expr > col_expr.mean(), col_expr, 0.0).execute() + + ser = batting_df['AB'] + expected = ser.where(ser > ser.mean(), other=0.0) + + tm.assert_series_equal(result.compute(), expected.compute()) + + +def test_round(t, df): + precision = 2 + mult = 3.33333 + result = (t.count() * mult).round(precision).execute() + expected = np.around(len(df) * mult, precision) + npt.assert_almost_equal(result, expected, decimal=precision) + + +def test_quantile_groupby(batting, batting_df): + def q_fun(x, quantile, interpolation): + res = x.quantile(quantile, interpolation=interpolation).tolist() + return [res for _ in range(len(x))] + + frac = 0.2 + intp = 'linear' + result = ( + batting.groupby('teamID') + .mutate(res=lambda x: x.RBI.quantile([frac, 1 - frac], intp)) + .res.execute() + ) + expected = ( + batting_df.groupby('teamID') + .RBI.transform(q_fun, quantile=[frac, 1 - frac], interpolation=intp) + .rename('res') + ) + tm.assert_series_equal(result.compute(), expected.compute()) + + +def test_summary_numeric(batting, batting_df): + expr = batting.G.summary() + result = expr.execute() + assert len(result) == 1 + + G = batting_df.G + expected = dict( + count=G.count(), + nulls=G.isnull().sum(), + min=G.min(), + max=G.max(), + sum=G.sum(), + mean=G.mean(), + approx_nunique=G.nunique(), + ) + assert dict(result.iloc[0]) == expected + + +def test_summary_numeric_group_by(batting, batting_df): + expr = batting.groupby('teamID').G.summary() + result = expr.execute() + expected = ( + batting_df.groupby('teamID') + .G.apply( + lambda s: dd.from_pandas( + pd.DataFrame( + dict( + count=s.count(), + nulls=s.isnull().sum(), + min=s.min(), + max=s.max(), + sum=s.sum(), + mean=s.mean(), + approx_nunique=s.nunique(), + ), + index=[0], + ), + npartitions=1, + ) + ) + .reset_index(level=1, drop=True) + .reset_index() + ) + columns = expected.columns + + tm.assert_frame_equal(result[columns], expected) + + +def test_summary_non_numeric(batting, batting_df): + expr = batting.teamID.summary() + result = expr.execute() + assert len(result) == 1 + assert len(result.columns) == 3 + expected = dict( + count=batting_df.teamID.count(), + nulls=batting_df.teamID.isnull().sum(), + uniques=batting_df.teamID.nunique(), + ) + assert dict(result.iloc[0]) == expected + + +def test_summary_non_numeric_group_by(batting, batting_df): + expr = batting.groupby('teamID').playerID.summary() + result = expr.execute() + expected = ( + batting_df.groupby('teamID') + .playerID.apply( + lambda s: dd.from_pandas( + pd.DataFrame( + dict( + count=s.count(), + nulls=s.isnull().sum(), + uniques=s.nunique(), + ), + index=[0], + ), + npartitions=1, + ) + ) + .reset_index(level=1, drop=True) + .reset_index() + ) + columns = expected.columns + tm.assert_frame_equal(result[columns], expected, check_dtype=False) + + +def test_searched_case_scalar(client): + expr = ibis.case().when(True, 1).when(False, 2).end() + result = client.execute(expr) + expected = np.int8(1) + assert result == expected + + +def test_searched_case_column(batting, batting_df): + t = batting + df = batting_df + expr = ( + ibis.case() + .when(t.RBI < 5, 'really bad team') + .when(t.teamID == 'PH1', 'ph1 team') + .else_(t.teamID) + .end() + ) + result = expr.execute() + expected = dd.from_array( + np.select( + [df.RBI < 5, df.teamID == 'PH1'], + ['really bad team', 'ph1 team'], + df.teamID, + ) + ) + tm.assert_series_equal(result.compute(), expected.compute()) + + +def test_simple_case_scalar(client): + x = ibis.literal(2) + expr = x.case().when(2, x - 1).when(3, x + 1).when(4, x + 2).end() + result = client.execute(expr) + expected = np.int8(1) + assert result == expected + + +def test_simple_case_column(batting, batting_df): + t = batting + df = batting_df + expr = ( + t.RBI.case() + .when(5, 'five') + .when(4, 'four') + .when(3, 'three') + .else_('could be good?') + .end() + ) + result = expr.execute() + expected = dd.from_array( + np.select( + [df.RBI == 5, df.RBI == 4, df.RBI == 3], + ['five', 'four', 'three'], + 'could be good?', + ) + ) + tm.assert_series_equal(result.compute(), expected.compute()) + + +def test_table_distinct(t, df): + expr = t[['dup_strings']].distinct() + result = expr.execute() + expected = df[['dup_strings']].drop_duplicates() + tm.assert_frame_equal(result.compute(), expected.compute()) + + +@pytest.mark.parametrize("distinct", [True, False]) +def test_union(client, df1, distinct): + t = client.table('df1') + expr = t.union(t, distinct=distinct) + result = expr.execute() + expected = ( + df1 if distinct else dd.concat([df1, df1], axis=0, ignore_index=True) + ) + tm.assert_frame_equal(result.compute(), expected.compute()) + + +def test_intersect(client, df1, intersect_df2): + t1 = client.table('df1') + t2 = client.table('intersect_df2') + expr = t1.intersect(t2) + result = expr.execute() + expected = df1.merge(intersect_df2, on=list(df1.columns)) + tm.assert_frame_equal(result.compute(), expected.compute()) + + +def test_difference(client, df1, intersect_df2): + t1 = client.table('df1') + t2 = client.table('intersect_df2') + expr = t1.difference(t2) + result = expr.execute() + merged = df1.merge( + intersect_df2, on=list(df1.columns), how="outer", indicator=True + ) + expected = merged[merged["_merge"] != "both"].drop("_merge", 1) + tm.assert_frame_equal(result.compute(), expected.compute()) + + +@pytest.mark.parametrize( + "distinct", + [ + pytest.param( + True, + marks=pytest.mark.xfail( + raises=TypeError, + reason=( + "dask cannot compute the distinct element of an " + "array column" + ), + ), + ), + False, + ], +) +def test_union_with_list_types(t, df, distinct): + expr = t.union(t, distinct=distinct) + result = expr.execute() + expected = ( + df if distinct else dd.concat([df, df], axis=0, ignore_index=True) + ) + tm.assert_frame_equal(result.compute(), expected.compute()) diff --git a/ibis/backends/dask/tests/execution/test_strings.py b/ibis/backends/dask/tests/execution/test_strings.py new file mode 100644 index 000000000000..d7c8435e93d1 --- /dev/null +++ b/ibis/backends/dask/tests/execution/test_strings.py @@ -0,0 +1,101 @@ +from warnings import catch_warnings + +import pytest +from dask.dataframe.utils import tm # noqa: E402 +from pytest import param + +pytestmark = pytest.mark.dask + + +@pytest.mark.parametrize( + ('case_func', 'expected_func'), + [ + param( + lambda s: s.length(), + lambda s: s.str.len().astype('int32'), + id='length', + ), + param(lambda s: s.substr(1, 2), lambda s: s.str[1:3], id='substr'), + param(lambda s: s[1:3], lambda s: s.str[1:3], id='slice'), + # TODO - execute_substring_series_series is broken + # param( + # lambda s: s[s.length() - 1 :], + # lambda s: s.str[-1:], + # id='expr_slice_begin', + # ), + # param(lambda s: s[: s.length()], lambda s: s, id='expr_slice_end'), + # param( + # lambda s: s[s.length() - 2 : s.length() - 1], + # lambda s: s.str[-2:-1], + # id='expr_slice_begin_end', + # ), + param(lambda s: s.strip(), lambda s: s.str.strip(), id='strip'), + param(lambda s: s.lstrip(), lambda s: s.str.lstrip(), id='lstrip'), + param(lambda s: s.rstrip(), lambda s: s.str.rstrip(), id='rstrip'), + param( + lambda s: s.lpad(3, 'a'), + lambda s: s.str.pad(3, side='left', fillchar='a'), + id='lpad', + ), + param( + lambda s: s.rpad(3, 'b'), + lambda s: s.str.pad(3, side='right', fillchar='b'), + id='rpad', + ), + param(lambda s: s.reverse(), lambda s: s.str[::-1], id='reverse'), + param(lambda s: s.lower(), lambda s: s.str.lower(), id='lower'), + param(lambda s: s.upper(), lambda s: s.str.upper(), id='upper'), + param( + lambda s: s.capitalize(), + lambda s: s.str.capitalize(), + id='capitalize', + ), + param(lambda s: s.repeat(2), lambda s: s * 2, id='repeat'), + param( + lambda s: s.contains('a'), + lambda s: s.str.contains('a', regex=False), + id='contains', + ), + param( + lambda s: ~(s.contains('a')), + lambda s: ~s.str.contains('a', regex=False), + id='not_contains', + ), + param( + lambda s: s.like('a'), + lambda s: s.str.contains('^a$', regex=True), + id='like', + ), + param( + lambda s: s.re_search('(ab)+'), + lambda s: s.str.contains('(ab)+', regex=True), + id='re_search', + ), + param( + lambda s: s.re_search('(ab)+') | s.re_search('d{1,2}ee'), + lambda s: ( + s.str.contains('(ab)+', regex=True) + | s.str.contains('d{1,2}ee') + ), + id='re_search_or', + ), + param( + lambda s: s + s.rpad(3, 'a'), + lambda s: s + s.str.pad(3, side='right', fillchar='a'), + id='rpad2', + ), + param( + lambda s: s.split(' '), + lambda s: s.str.split(' '), + id='split_spaces', + ), + ], +) +def test_string_ops(t, df, case_func, expected_func): + + # ignore matching UserWarnings + with catch_warnings(record=True): + expr = case_func(t.strings_with_space) + result = expr.execute() + series = expected_func(df.strings_with_space) + tm.assert_series_equal(result.compute(), series.compute()) diff --git a/ibis/backends/dask/tests/execution/test_structs.py b/ibis/backends/dask/tests/execution/test_structs.py new file mode 100644 index 000000000000..efd7f3ff73cc --- /dev/null +++ b/ibis/backends/dask/tests/execution/test_structs.py @@ -0,0 +1,96 @@ +from collections import OrderedDict + +import dask.dataframe as dd +import pandas as pd +import pytest +from dask.dataframe.utils import tm + +import ibis +import ibis.expr.datatypes as dt + +from ... import connect, execute + + +@pytest.fixture(scope="module") +def value(): + return OrderedDict([("fruit", "pear"), ("weight", 0)]) + + +@pytest.fixture(scope="module") +def struct_client(value): + df = dd.from_pandas( + pd.DataFrame( + { + "s": [ + OrderedDict([("fruit", "apple"), ("weight", None)]), + value, + OrderedDict([("fruit", "pear"), ("weight", 1)]), + ], + "key": list("aab"), + "value": [1, 2, 3], + } + ), + npartitions=1, + ) + return connect({"t": df}) + + +@pytest.fixture +def struct_table(struct_client): + return struct_client.table( + "t", + schema={ + "s": dt.Struct.from_tuples( + [("fruit", dt.string), ("weight", dt.int8)] + ) + }, + ) + + +def test_struct_field_literal(value): + struct = ibis.literal(value) + assert struct.type() == dt.Struct.from_tuples( + [("fruit", dt.string), ("weight", dt.int8)] + ) + + expr = struct.fruit + result = execute(expr) + assert result == "pear" + + expr = struct.weight + result = execute(expr) + assert result == 0 + + +def test_struct_field_series(struct_table): + t = struct_table + expr = t.s.fruit + result = expr.execute() + expected = dd.from_pandas( + pd.Series(["apple", "pear", "pear"], name="fruit"), npartitions=1, + ) + tm.assert_series_equal(result.compute(), expected.compute()) + + +def test_struct_field_series_group_by_key(struct_table): + t = struct_table + expr = t.groupby(t.s.fruit).aggregate(total=t.value.sum()) + result = expr.execute() + expected = dd.from_pandas( + pd.DataFrame([("apple", 1), ("pear", 5)], columns=["fruit", "total"]), + npartitions=1, + ) + tm.assert_frame_equal(result.compute(), expected.compute()) + + +@pytest.mark.xfail(reason="TODO - grouping - #2553") +def test_struct_field_series_group_by_value(struct_table): + t = struct_table + expr = t.groupby(t.key).aggregate(total=t.s.weight.sum()) + result = expr.execute() + # these are floats because we have a NULL value in the input data + expected = dd.from_pandas( + pd.DataFrame([("a", 0.0), ("b", 1.0)], columns=["key", "total"]), + npartitions=1, + ) + tm.assert_frame_equal(result.compute(), expected.compute()) diff --git a/ibis/backends/dask/tests/execution/test_temporal.py b/ibis/backends/dask/tests/execution/test_temporal.py new file mode 100644 index 000000000000..c8f959f67b08 --- /dev/null +++ b/ibis/backends/dask/tests/execution/test_temporal.py @@ -0,0 +1,199 @@ +import datetime +from operator import methodcaller + +import dask.dataframe as dd +import numpy as np +import pandas as pd +import pytest +from dask.dataframe.utils import tm # noqa: E402 +from pkg_resources import parse_version +from pytest import param + +from ibis import literal as L # noqa: E402 +from ibis.expr import datatypes as dt + +from ... import connect, execute + +pytestmark = pytest.mark.dask + + +@pytest.mark.parametrize( + ('case_func', 'expected_func'), + [ + (lambda v: v.strftime('%Y%m%d'), lambda vt: vt.strftime('%Y%m%d')), + (lambda v: v.year(), lambda vt: vt.year), + (lambda v: v.month(), lambda vt: vt.month), + (lambda v: v.day(), lambda vt: vt.day), + (lambda v: v.hour(), lambda vt: vt.hour), + (lambda v: v.minute(), lambda vt: vt.minute), + (lambda v: v.second(), lambda vt: vt.second), + (lambda v: v.millisecond(), lambda vt: int(vt.microsecond / 1e3)), + ] + + [ + (methodcaller('strftime', pattern), methodcaller('strftime', pattern)) + for pattern in [ + '%Y%m%d %H', + 'DD BAR %w FOO "DD"', + 'DD BAR %w FOO "D', + 'DD BAR "%w" FOO "D', + 'DD BAR "%d" FOO "D', + 'DD BAR "%c" FOO "D', + 'DD BAR "%x" FOO "D', + 'DD BAR "%X" FOO "D', + ] + ], +) +def test_timestamp_functions(case_func, expected_func): + v = L('2015-09-01 14:48:05.359').cast('timestamp') + vt = datetime.datetime( + year=2015, + month=9, + day=1, + hour=14, + minute=48, + second=5, + microsecond=359000, + ) + result = case_func(v) + expected = expected_func(vt) + assert execute(result) == expected + + +@pytest.mark.parametrize( + 'column', + ['datetime_strings_naive', 'datetime_strings_ny', 'datetime_strings_utc'], +) +def test_cast_datetime_strings_to_date(t, df, column): + # TODO - this is changed from the pandas test, double check + expr = t[column].cast('date') + result = expr.execute() + df_computed = df.compute() + expected = dd.from_pandas( + pd.to_datetime( + df_computed[column], infer_datetime_format=True, + ).dt.normalize(), + npartitions=1, + ) + tm.assert_series_equal(result.compute(), expected.compute()) + + +@pytest.mark.parametrize( + 'column', + ['datetime_strings_naive', 'datetime_strings_ny', 'datetime_strings_utc'], +) +def test_cast_datetime_strings_to_timestamp(t, df, column): + expr = t[column].cast('timestamp') + result = expr.execute() + df_computed = df.compute() + expected = dd.from_pandas( + pd.to_datetime(df_computed[column], infer_datetime_format=True), + npartitions=1, + ) + if getattr(expected.dtype, 'tz', None) is not None: + expected = expected.dt.tz_convert(None) + tm.assert_series_equal(result.compute(), expected.compute()) + + +@pytest.mark.parametrize( + 'column', + ['plain_datetimes_naive', 'plain_datetimes_ny', 'plain_datetimes_utc'], +) +def test_cast_integer_to_temporal_type(t, df, column): + column_type = t[column].type() + expr = t.plain_int64.cast(column_type) + result = expr.execute() + df_computed = df.compute() + expected = dd.from_pandas( + pd.Series( + pd.to_datetime(df_computed.plain_int64.values, unit='ns').values, + index=df_computed.index, + name='plain_int64', + ).dt.tz_localize(column_type.timezone), + npartitions=1, + ) + tm.assert_series_equal(result.compute(), expected.compute()) + + +def test_cast_integer_to_date(t, df): + expr = t.plain_int64.cast('date') + result = expr.execute() + df_computed = df.compute() + expected = dd.from_pandas( + pd.Series( + pd.to_datetime(df_computed.plain_int64.values, unit='D').values, + index=df_computed.index, + name='plain_int64', + ), + npartitions=1, + ) + tm.assert_series_equal(result.compute(), expected.compute()) + + +def test_times_ops(t, df): + result = t.plain_datetimes_naive.time().between('10:00', '10:00').execute() + expected = dd.from_array(np.zeros(len(df), dtype=bool)) + tm.assert_series_equal(result.compute(), expected.compute()) + + result = t.plain_datetimes_naive.time().between('01:00', '02:00').execute() + expected = dd.from_array(np.ones(len(df), dtype=bool)) + tm.assert_series_equal(result.compute(), expected.compute()) + + +@pytest.mark.parametrize( + ('tz', 'rconstruct'), + [('US/Eastern', np.zeros), ('UTC', np.ones), (None, np.ones)], +) +@pytest.mark.parametrize( + 'column', ['plain_datetimes_utc', 'plain_datetimes_naive'] +) +def test_times_ops_with_tz(t, df, tz, rconstruct, column): + expected = dd.from_array(rconstruct(len(df), dtype=bool),) + time = t[column].time() + expr = time.between('01:00', '02:00', timezone=tz) + result = expr.execute() + tm.assert_series_equal(result.compute(), expected.compute()) + + # Test that casting behavior is the same as using the timezone kwarg + ts = t[column].cast(dt.Timestamp(timezone=tz)) + expr = ts.time().between('01:00', '02:00') + result = expr.execute() + tm.assert_series_equal(result.compute(), expected.compute()) + + +@pytest.mark.parametrize( + ('op', 'expected'), + [ + param(lambda x, y: x + y, lambda x, y: x.values * 2, id='add'), + param(lambda x, y: x - y, lambda x, y: x.values - y.values, id='sub'), + param(lambda x, y: x * 2, lambda x, y: x.values * 2, id='mul'), + param( + lambda x, y: x // 2, + lambda x, y: x.values // 2, + id='floordiv', + marks=pytest.mark.xfail( + parse_version(pd.__version__) < parse_version('0.23.0'), + raises=TypeError, + reason=( + 'pandas versions less than 0.23.0 do not support floor ' + 'division involving timedelta columns' + ), + ), + ), + ], +) +def test_interval_arithmetic(op, expected): + data = pd.timedelta_range('0 days', '10 days', freq='D') + pandas_df = pd.DataFrame({'td': data}) + con = connect( + { + 'df1': dd.from_pandas(pandas_df, npartitions=1), + 'df2': dd.from_pandas(pandas_df, npartitions=1), + } + ) + t1 = con.table('df1') + expr = op(t1.td, t1.td) + result = expr.execute() + expected = dd.from_pandas( + pd.Series(expected(data, data), name='td'), npartitions=1, + ) + tm.assert_series_equal(result.compute(), expected.compute()) diff --git a/ibis/backends/dask/tests/execution/test_timecontext.py b/ibis/backends/dask/tests/execution/test_timecontext.py new file mode 100644 index 000000000000..e439ab9041a0 --- /dev/null +++ b/ibis/backends/dask/tests/execution/test_timecontext.py @@ -0,0 +1,222 @@ +import dask.dataframe as dd +import pytest +from dask.dataframe.utils import tm +from pandas import Timedelta, Timestamp + +import ibis +import ibis.common.exceptions as com +from ibis.expr.timecontext import TimeContextRelation, compare_timecontext + +pytestmark = pytest.mark.dask + + +def test_execute_with_timecontext(time_table): + expr = time_table + # define a time context for time-series data + context = (Timestamp('20170101'), Timestamp('20170103')) + + # without time context, execute produces every row + df_all = expr.execute() + assert len(df_all['time']) == 8 + + # with context set, execute produces only rows within context + df_within_context = expr.execute(timecontext=context) + assert len(df_within_context['time']) == 1 + + +def test_bad_timecontext(time_table, t): + expr = time_table + + # define context with illegal string + with pytest.raises(com.IbisError, match=r".*type pd.Timestamp.*"): + context = ('bad', 'context') + expr.execute(timecontext=context) + + # define context with unsupport type int + with pytest.raises(com.IbisError, match=r".*type pd.Timestamp.*"): + context = (20091010, 20100101) + expr.execute(timecontext=context) + + # define context with too few values + with pytest.raises(com.IbisError, match=r".*should specify.*"): + context = Timestamp('20101010') + expr.execute(timecontext=context) + + # define context with begin value later than end + with pytest.raises(com.IbisError, match=r".*before or equal.*"): + context = (Timestamp('20101010'), Timestamp('20090101')) + expr.execute(timecontext=context) + + # execute context with a table without TIME_COL + with pytest.raises(com.IbisError, match=r".*must have a time column.*"): + context = (Timestamp('20090101'), Timestamp('20100101')) + t.execute(timecontext=context) + + +def test_compare_timecontext(): + c1 = (Timestamp('20170101'), Timestamp('20170103')) + c2 = (Timestamp('20170101'), Timestamp('20170111')) + c3 = (Timestamp('20160101'), Timestamp('20160103')) + c4 = (Timestamp('20161215'), Timestamp('20170102')) + assert compare_timecontext(c1, c2) == TimeContextRelation.SUBSET + assert compare_timecontext(c2, c1) == TimeContextRelation.SUPERSET + assert compare_timecontext(c1, c4) == TimeContextRelation.OVERLAP + assert compare_timecontext(c1, c3) == TimeContextRelation.NONOVERLAP + + +def test_context_adjustment_asof_join( + time_keyed_left, time_keyed_right, time_keyed_df1, time_keyed_df2 +): + expr = time_keyed_left.asof_join( + time_keyed_right, 'time', by='key', tolerance=4 * ibis.interval(days=1) + )[time_keyed_left, time_keyed_right.other_value] + context = (Timestamp('20170105'), Timestamp('20170111')) + result = expr.execute(timecontext=context) + + # compare with asof_join of manually trimmed tables + trimmed_df1 = time_keyed_df1[time_keyed_df1['time'] >= context[0]][ + time_keyed_df1['time'] < context[1] + ] + trimmed_df2 = time_keyed_df2[ + time_keyed_df2['time'] >= context[0] - Timedelta(days=4) + ][time_keyed_df2['time'] < context[1]] + expected = dd.merge_asof( + trimmed_df1, + trimmed_df2, + on='time', + by='key', + tolerance=Timedelta('4D'), + ) + tm.assert_frame_equal(result.compute(), expected.compute()) + + +@pytest.mark.xfail(reason="TODO - windowing - #2553") +@pytest.mark.parametrize( + ['interval_ibis', 'interval_pd'], + [ + (ibis.interval(days=1), '1d'), + (3 * ibis.interval(days=1), '3d'), + (5 * ibis.interval(days=1), '5d'), + ], +) +def test_context_adjustment_window( + time_table, time_df3, interval_ibis, interval_pd +): + # trim data manually + expected = ( + time_df3.set_index('time') + .value.rolling(interval_pd, closed='both') + .mean() + ) + expected = expected[expected.index >= Timestamp('20170105')].reset_index( + drop=True + ) + + context = Timestamp('20170105'), Timestamp('20170111') + + window = ibis.trailing_window(interval_ibis, order_by=time_table.time) + expr = time_table['value'].mean().over(window) + # result should adjust time context accordingly + result = expr.execute(timecontext=context) + tm.assert_series_equal(result, expected) + + +@pytest.mark.xfail(reason="TODO - windowing - #2553") +def test_setting_timecontext_in_scope(time_table, time_df3): + expected_win_1 = ( + time_df3.compute() + .set_index('time') + .value.rolling('3d', closed='both') + .mean() + ) + expected_win_1 = expected_win_1[ + expected_win_1.index >= Timestamp('20170105') + ].reset_index(drop=True) + + context = Timestamp('20170105'), Timestamp('20170111') + window1 = ibis.trailing_window( + 3 * ibis.interval(days=1), order_by=time_table.time + ) + """ + In the following expression, Selection node will be executed first and + get table in context ('20170105', '20170101'). Then in window execution + table will be executed again with a larger context adjusted by window + preceeding days ('20170102', '20170111'). To get the correct result, + the cached table result with a smaller context must be discard and updated + to a larger time range. + """ + expr = time_table.mutate(value=time_table['value'].mean().over(window1)) + result = expr.execute(timecontext=context) + tm.assert_series_equal(result["value"], expected_win_1) + + +@pytest.mark.xfail(reason="TODO - windowing - #2553") +def test_context_adjustment_multi_window(time_table, time_df3): + expected_win_1 = ( + time_df3.compute() + .set_index('time') + .rename(columns={'value': 'v1'})['v1'] + .rolling('3d', closed='both') + .mean() + ) + expected_win_1 = expected_win_1[ + expected_win_1.index >= Timestamp('20170105') + ].reset_index(drop=True) + + expected_win_2 = ( + time_df3.compute() + .set_index('time') + .rename(columns={'value': 'v2'})['v2'] + .rolling('2d', closed='both') + .mean() + ) + expected_win_2 = expected_win_2[ + expected_win_2.index >= Timestamp('20170105') + ].reset_index(drop=True) + + context = Timestamp('20170105'), Timestamp('20170111') + window1 = ibis.trailing_window( + 3 * ibis.interval(days=1), order_by=time_table.time + ) + window2 = ibis.trailing_window( + 2 * ibis.interval(days=1), order_by=time_table.time + ) + expr = time_table.mutate( + v1=time_table['value'].mean().over(window1), + v2=time_table['value'].mean().over(window2), + ) + result = expr.execute(timecontext=context) + + tm.assert_series_equal(result["v1"], expected_win_1) + tm.assert_series_equal(result["v2"], expected_win_2) + + +@pytest.mark.xfail(reason="TODO - windowing - #2553") +def test_context_adjustment_window_groupby_id(time_table, time_df3): + """ This test case is meant to test trim_with_timecontext method + in dask/execution/window.py to see if it could trim Series + correctly with groupby params + """ + expected = ( + time_df3.compute() + .set_index('time') + .groupby('id') + .value.rolling('3d', closed='both') + .mean() + ) + # This is a MultiIndexed Series + expected = expected.reset_index() + expected = expected[expected.time >= Timestamp('20170105')].reset_index( + drop=True + )['value'] + + context = Timestamp('20170105'), Timestamp('20170111') + + # expected.index.name = None + window = ibis.trailing_window( + 3 * ibis.interval(days=1), group_by='id', order_by=time_table.time + ) + expr = time_table['value'].mean().over(window) + # result should adjust time context accordingly + result = expr.execute(timecontext=context) + tm.assert_series_equal(result, expected) diff --git a/ibis/backends/dask/tests/test_client.py b/ibis/backends/dask/tests/test_client.py index b6ecb131ec43..4318106a5a56 100644 --- a/ibis/backends/dask/tests/test_client.py +++ b/ibis/backends/dask/tests/test_client.py @@ -54,7 +54,6 @@ def test_load_data(client): assert client.get_schema('testing') -@pytest.mark.xfail(reason="Need to fix up create_table") def test_create_table(client): client.create_table('testing', obj=make_dask_data_frame()) assert client.exists_table('testing') @@ -79,7 +78,6 @@ def test_read_with_undiscoverable_type(client): client.table('df_unknown') -@pytest.mark.xfail(reason="TODO remove when execution is written") def test_drop(table): table = table.mutate(c=table.a) expr = table.drop(['a']) diff --git a/ibis/backends/dask/tests/test_core.py b/ibis/backends/dask/tests/test_core.py index cb55dc95d0f2..c7833bf89cd5 100644 --- a/ibis/backends/dask/tests/test_core.py +++ b/ibis/backends/dask/tests/test_core.py @@ -28,7 +28,6 @@ def test_no_execute_ambiguities(func): assert not ambiguities(func.funcs) -@pytest.mark.xfail(reason="TODO remove when execution is written") def test_from_dataframe(dataframe, ibis_table, core_client): t = from_dataframe(dataframe) result = t.execute() @@ -97,7 +96,6 @@ def table(self, name): con.execute(t) -@pytest.mark.xfail(reason="TODO remove when execution is written") def test_post_execute_called_on_joins(dataframe, core_client, ibis_table): count = [0] diff --git a/ibis/backends/dask/tests/test_datatypes.py b/ibis/backends/dask/tests/test_datatypes.py index 814d805af08d..065d9c312ac4 100644 --- a/ibis/backends/dask/tests/test_datatypes.py +++ b/ibis/backends/dask/tests/test_datatypes.py @@ -85,7 +85,7 @@ def test_dask_dtype(dask_dtype, ibis_dtype): assert dt.dtype(dask_dtype) == ibis_dtype -@pytest.mark.xfail(reason="TODO - literal conversion does not work yet") +@pytest.mark.xfail(TypeError, reason="TODO - as_value_expr - #2553") def test_series_to_ibis_literal(): values = [1, 2, 3, 4] s = dd.from_pandas(pd.Series(values), npartitions=1) diff --git a/ibis/tests/all/conftest.py b/ibis/tests/all/conftest.py index 962d5f0b92aa..b057a47fd4ef 100644 --- a/ibis/tests/all/conftest.py +++ b/ibis/tests/all/conftest.py @@ -8,7 +8,7 @@ import ibis import ibis.common.exceptions as com import ibis.util as util -from ibis.tests.backends import Backend +from ibis.tests.backends import Backend, Dask def _random_identifier(suffix): @@ -157,7 +157,9 @@ def alltypes(backend): @pytest.fixture(scope='session') -def sorted_alltypes(alltypes): +def sorted_alltypes(backend, alltypes): + if isinstance(backend, Dask): + pytest.skip("# TODO - sorting - #2553") return alltypes.sort_by('id') @@ -193,7 +195,10 @@ def df(alltypes): @pytest.fixture(scope='session') -def sorted_df(df): +def sorted_df(backend, df): + if isinstance(backend, Dask): + pytest.skip("# TODO - sorting - #2553") + return df.sort_values('id').reset_index(drop=True) diff --git a/ibis/tests/all/test_aggregation.py b/ibis/tests/all/test_aggregation.py index f779a1fe2dac..c9b943bd22ec 100644 --- a/ibis/tests/all/test_aggregation.py +++ b/ibis/tests/all/test_aggregation.py @@ -7,6 +7,7 @@ from ibis.tests.backends import ( BigQuery, Clickhouse, + Dask, MySQL, Postgres, PySpark, @@ -64,6 +65,9 @@ def mean_udf(s): @pytest.mark.parametrize( ('result_fn', 'expected_fn', 'expected_col'), aggregate_test_params, ) +@pytest.mark.skip_backends( + [Dask] +) # TODO - aggregations - #2553 (and pd.concat) @pytest.mark.xfail_unsupported def test_aggregate( backend, alltypes, df, result_fn, expected_fn, expected_col @@ -81,6 +85,7 @@ def test_aggregate( @pytest.mark.parametrize( ('result_fn', 'expected_fn', 'expected_col'), aggregate_test_params, ) +@pytest.mark.skip_backends([Dask]) # TODO - aggregations - #2553 @pytest.mark.xfail_unsupported def test_aggregate_grouped( backend, alltypes, df, result_fn, expected_fn, expected_col @@ -235,6 +240,7 @@ def test_aggregate_grouped( ), ], ) +@pytest.mark.skip_backends([Dask]) # TODO - iloc - #2553 @pytest.mark.xfail_unsupported def test_reduction_ops( backend, alltypes, df, result_fn, expected_fn, ibis_cond, pandas_cond @@ -264,6 +270,7 @@ def test_reduction_ops( ) ], ) +@pytest.mark.skip_backends([Dask]) # TODO - aggregations - #2553 @pytest.mark.xfail_unsupported def test_group_concat(backend, alltypes, df, result_fn, expected_fn): expr = result_fn(alltypes) @@ -284,7 +291,8 @@ def test_group_concat(backend, alltypes, df, result_fn, expected_fn): ], ) @pytest.mark.xfail_unsupported -@pytest.mark.xfail_backends([PySpark]) # Issue #2130 +# TODO - sorting - #2553 +@pytest.mark.xfail_backends([Dask, PySpark]) # Issue #2130 def test_topk_op(backend, alltypes, df, result_fn, expected_fn): # TopK expression will order rows by "count" but each backend # can have different result for that. @@ -313,7 +321,8 @@ def test_topk_op(backend, alltypes, df, result_fn, expected_fn): ) @pytest.mark.xfail_unsupported # Issues #2369 #2133 #2131 #2132 -@pytest.mark.xfail_backends([BigQuery, Clickhouse, MySQL, Postgres]) +# TODO - sorting - #2553 +@pytest.mark.xfail_backends([BigQuery, Clickhouse, Dask, MySQL, Postgres]) @pytest.mark.skip_backends([SQLite], reason='Issue #2128') def test_topk_filter_op(backend, alltypes, df, result_fn, expected_fn): # TopK expression will order rows by "count" but each backend diff --git a/ibis/tests/all/test_generic.py b/ibis/tests/all/test_generic.py index 726b11e986bb..0622b572913e 100644 --- a/ibis/tests/all/test_generic.py +++ b/ibis/tests/all/test_generic.py @@ -5,6 +5,7 @@ import ibis from ibis import literal as L +from ibis.tests.backends import Dask @pytest.mark.parametrize( @@ -51,7 +52,8 @@ def test_coalesce(backend, con, expr, expected): @pytest.mark.xfail_unsupported -def test_identical_to(backend, sorted_alltypes, con, sorted_df): +def test_identical_to(backend, alltypes, con, sorted_df): + sorted_alltypes = alltypes.sort_by('id') df = sorted_df dt = df[['tinyint_col', 'double_col']] @@ -80,8 +82,10 @@ def test_identical_to(backend, sorted_alltypes, con, sorted_df): ('int_col', frozenset({1})), ], ) +@pytest.mark.skip_backends([Dask]) # TODO - sorting - #2553 @pytest.mark.xfail_unsupported -def test_isin(backend, sorted_alltypes, sorted_df, column, elements): +def test_isin(backend, alltypes, sorted_df, column, elements): + sorted_alltypes = alltypes.sort_by('id') expr = sorted_alltypes[ 'id', sorted_alltypes[column].isin(elements).name('tmp') ].sort_by('id') @@ -103,8 +107,10 @@ def test_isin(backend, sorted_alltypes, sorted_df, column, elements): ('int_col', frozenset({1})), ], ) +@pytest.mark.skip_backends([Dask]) # TODO - sorting - #2553 @pytest.mark.xfail_unsupported -def test_notin(backend, sorted_alltypes, sorted_df, column, elements): +def test_notin(backend, alltypes, sorted_df, column, elements): + sorted_alltypes = alltypes.sort_by('id') expr = sorted_alltypes[ 'id', sorted_alltypes[column].notin(elements).name('tmp') ].sort_by('id') diff --git a/ibis/tests/all/test_join.py b/ibis/tests/all/test_join.py index 819f6fc37694..a8976e9a2cf3 100644 --- a/ibis/tests/all/test_join.py +++ b/ibis/tests/all/test_join.py @@ -2,7 +2,7 @@ import pytest from pytest import param -from ibis.tests.backends import Csv, Pandas, PySpark +from ibis.tests.backends import Csv, Dask, Pandas, PySpark # add here backends that passes join tests all_db_join_supported = [Pandas, PySpark] @@ -31,7 +31,8 @@ ) @pytest.mark.only_on_backends(all_db_join_supported) # Csv is a subclass of Pandas so need to skip it explicitly. -@pytest.mark.skip_backends([Csv]) +# TODO - sorting - #2553 +@pytest.mark.skip_backends([Csv, Dask]) @pytest.mark.xfail_unsupported def test_join_project_left_table(backend, con, batting, awards_players, how): diff --git a/ibis/tests/all/test_numeric.py b/ibis/tests/all/test_numeric.py index 02c06edffb14..cced1d777bde 100644 --- a/ibis/tests/all/test_numeric.py +++ b/ibis/tests/all/test_numeric.py @@ -99,7 +99,10 @@ def test_isnan_isinf( expected = backend.default_series_rename(expected) backend.assert_series_equal(result, expected) else: - assert result == expected + try: + assert result == expected + except ValueError: + backend.assert_series_equal(result, expected) @pytest.mark.parametrize( diff --git a/ibis/tests/all/test_string.py b/ibis/tests/all/test_string.py index 03acfcbe4bfa..c62e5f93afed 100644 --- a/ibis/tests/all/test_string.py +++ b/ibis/tests/all/test_string.py @@ -6,6 +6,7 @@ from ibis.tests.backends import ( BigQuery, Clickhouse, + Dask, Impala, OmniSciDB, Postgres, @@ -14,6 +15,7 @@ ) +@pytest.mark.xfail_backends([Dask]) # TODO - pandas - #2553 def test_string_col_is_unicode(backend, alltypes, df): dtype = alltypes.string_col.type() assert dtype == dt.String(nullable=dtype.nullable) @@ -159,6 +161,8 @@ def test_string_col_is_unicode(backend, alltypes, df): lambda t: t.string_col.ascii_str(), lambda t: t.string_col.map(ord).astype('int32'), id='ascii_str', + # TODO - pandas - #2553 + marks=pytest.mark.skip_backends([Dask]), ), param( lambda t: t.string_col.length(), @@ -210,16 +214,22 @@ def test_string_col_is_unicode(backend, alltypes, df): lambda t: t.date_string_col[t.date_string_col.length() - 1 :], lambda t: t.date_string_col.str[-1:], id='expr_slice_begin', + # TODO - substring - #2553 + marks=pytest.mark.xfail_backends([Dask]), ), param( lambda t: t.date_string_col[: t.date_string_col.length()], lambda t: t.date_string_col, id='expr_slice_end', + # TODO - substring - #2553 + marks=pytest.mark.xfail_backends([Dask]), ), param( lambda t: t.date_string_col[:], lambda t: t.date_string_col, id='expr_empty_slice', + # TODO - substring - #2553 + marks=pytest.mark.xfail_backends([Dask]), ), param( lambda t: t.date_string_col[ @@ -227,6 +237,8 @@ def test_string_col_is_unicode(backend, alltypes, df): ], lambda t: t.date_string_col.str[-2:-1], id='expr_slice_begin_end', + # TODO - substring - #2553 + marks=pytest.mark.xfail_backends([Dask]), ), param( lambda t: t.date_string_col.split('/'), diff --git a/ibis/tests/all/test_temporal.py b/ibis/tests/all/test_temporal.py index be6625190584..ee39b4f02130 100644 --- a/ibis/tests/all/test_temporal.py +++ b/ibis/tests/all/test_temporal.py @@ -13,6 +13,7 @@ BigQuery, Clickhouse, Csv, + Dask, Impala, Pandas, Parquet, @@ -97,6 +98,7 @@ def test_timestamp_extract(backend, alltypes, df, attr): 'ns', ], ) +@pytest.mark.skip_backends([Dask]) # TODO - pandas - #2553 @pytest.mark.xfail_unsupported def test_timestamp_truncate(backend, alltypes, df, unit): expr = alltypes.timestamp_col.truncate(unit) @@ -119,6 +121,7 @@ def test_timestamp_truncate(backend, alltypes, df, unit): param('W', marks=pytest.mark.xpass_backends((Csv, Pandas, Parquet))), ], ) +@pytest.mark.skip_backends([Dask]) # TODO - pandas - #2553 @pytest.mark.xfail_unsupported def test_date_truncate(backend, alltypes, df, unit): expr = alltypes.timestamp_col.date().truncate(unit) @@ -135,11 +138,31 @@ def test_date_truncate(backend, alltypes, df, unit): @pytest.mark.parametrize( ('unit', 'displacement_type'), [ - ('Y', pd.offsets.DateOffset), + param( + 'Y', + pd.offsets.DateOffset, + # TODO - DateOffset - #2553 + marks=pytest.mark.xfail_backends([Dask]), + ), param('Q', pd.offsets.DateOffset, marks=pytest.mark.xfail), - ('M', pd.offsets.DateOffset), - ('W', pd.offsets.DateOffset), - ('D', pd.offsets.DateOffset), + param( + 'M', + pd.offsets.DateOffset, + # TODO - DateOffset - #2553 + marks=pytest.mark.xfail_backends([Dask]), + ), + param( + 'W', + pd.offsets.DateOffset, + # TODO - DateOffset - #2553 + marks=pytest.mark.xfail_backends([Dask]), + ), + param( + 'D', + pd.offsets.DateOffset, + # TODO - DateOffset - #2553 + marks=pytest.mark.xfail_backends([Dask]), + ), ('h', pd.Timedelta), ('m', pd.Timedelta), ('s', pd.Timedelta), @@ -185,7 +208,8 @@ def convert_to_offset(offset, displacement_type=displacement_type): 'unit', ['Y', param('Q', marks=pytest.mark.xfail), 'M', 'W', 'D'] ) @pytest.mark.xfail_unsupported -@pytest.mark.skip_backends([Spark]) +# TODO - DateOffset - #2553 +@pytest.mark.skip_backends([Dask, Spark]) def test_integer_to_interval_date(backend, con, alltypes, df, unit): interval = alltypes.int_col.to_interval(unit=unit) array = alltypes.date_string_col.split('/') @@ -263,7 +287,8 @@ def test_integer_to_interval_date_failure(backend, con, alltypes, df, unit): ) ), id='timestamp-subtract-timestamp', - marks=pytest.mark.xfail_backends([Spark]), + # TODO - pandas - #2553 + marks=pytest.mark.xfail_backends([Dask, Spark]), ), param( lambda t, be: t.timestamp_col.date() - ibis.date(date_value), @@ -296,8 +321,9 @@ def test_interval_add_cast_scalar(backend, alltypes): @pytest.mark.xfail_unsupported +# TODO - pandas - #2553 # PySpark does not support casting columns to intervals -@pytest.mark.xfail_backends([PySpark]) +@pytest.mark.xfail_backends([Dask, PySpark]) @pytest.mark.skip_backends([Spark]) def test_interval_add_cast_column(backend, alltypes, df): timestamp_date = alltypes.timestamp_col.date() @@ -420,6 +446,8 @@ def test_day_of_week_column(backend, con, alltypes, df): ), ], ) +# TODO - grouping - #2553 +@pytest.mark.xfail_backends([Dask]) @pytest.mark.xfail_unsupported def test_day_of_week_column_group_by( backend, con, alltypes, df, day_of_week_expr, day_of_week_pandas @@ -454,6 +482,8 @@ def test_now(backend, con): assert result.year == pandas_now.year +# TODO - pandas - #2553 +@pytest.mark.xfail_backends([Dask]) @pytest.mark.xfail_unsupported def test_now_from_projection(backend, con, alltypes, df): n = 5 diff --git a/ibis/tests/all/test_union.py b/ibis/tests/all/test_union.py index 01c1ca9a7601..02cca430179f 100644 --- a/ibis/tests/all/test_union.py +++ b/ibis/tests/all/test_union.py @@ -1,11 +1,19 @@ import pandas as pd import pytest -from ibis.tests.backends import BigQuery, Impala, Pandas, Postgres, PySpark +from ibis.tests.backends import ( + BigQuery, + Dask, + Impala, + Pandas, + Postgres, + PySpark, +) @pytest.mark.parametrize('distinct', [False, True]) @pytest.mark.only_on_backends([BigQuery, Impala, Pandas, Postgres, PySpark]) +@pytest.mark.skip_backends([Dask]) # TODO - sorting - #2553 (pd.concat) @pytest.mark.xfail_unsupported def test_union(backend, alltypes, df, distinct): result = alltypes.union(alltypes, distinct=distinct).execute() diff --git a/ibis/tests/all/test_vectorized_udf.py b/ibis/tests/all/test_vectorized_udf.py index fe7f40f63f83..087e7c10c7bb 100644 --- a/ibis/tests/all/test_vectorized_udf.py +++ b/ibis/tests/all/test_vectorized_udf.py @@ -4,7 +4,7 @@ import ibis.common.exceptions as com import ibis.expr.datatypes as dt from ibis.expr.window import window -from ibis.tests.backends import Pandas, PySpark +from ibis.tests.backends import Dask, Pandas, PySpark from ibis.udf.vectorized import analytic, elementwise, reduction pytestmark = pytest.mark.udf @@ -234,6 +234,8 @@ def foo1(v, amount): @pytest.mark.only_on_backends([Pandas, PySpark]) +# TODO - udf - #2553 +@pytest.mark.xfail_backends([Dask]) @pytest.mark.xfail_unsupported def test_elementwise_udf_destruct(backend, alltypes): result = alltypes.mutate( @@ -279,6 +281,8 @@ def test_elementwise_udf_struct(backend, alltypes): @pytest.mark.only_on_backends([Pandas]) +# TODO - udf - #2553 +@pytest.mark.xfail_backends([Dask]) def test_analytic_udf_destruct(backend, alltypes): w = window(preceding=None, following=None, group_by='year') @@ -297,6 +301,8 @@ def test_analytic_udf_destruct(backend, alltypes): @pytest.mark.only_on_backends([Pandas]) +# TODO - udf - #2553 +@pytest.mark.xfail_backends([Dask]) def test_reduction_udf_destruct_groupby(backend, alltypes): result = ( alltypes.groupby('year') @@ -321,6 +327,8 @@ def test_reduction_udf_destruct_groupby(backend, alltypes): @pytest.mark.only_on_backends([Pandas]) +# TODO - udf - #2553 +@pytest.mark.xfail_backends([Dask]) def test_reduction_udf_destruct_no_groupby(backend, alltypes): result = alltypes.aggregate( mean_struct(alltypes['double_col'], alltypes['int_col']).destructure() @@ -335,6 +343,8 @@ def test_reduction_udf_destruct_no_groupby(backend, alltypes): @pytest.mark.only_on_backends([Pandas]) +# TODO - udf - #2553 +@pytest.mark.xfail_backends([Dask]) def test_reduction_udf_destruct_window(backend, alltypes): win = window( preceding=ibis.interval(hours=2), diff --git a/ibis/tests/backends.py b/ibis/tests/backends.py index 640379038f64..e88c5fc3e0cc 100644 --- a/ibis/tests/backends.py +++ b/ibis/tests/backends.py @@ -180,8 +180,61 @@ def connect(data_directory: Path) -> ibis.client.Client: class Dask(Pandas): - # clone pandas directly until the rest of the dask backend is defined - pass + @staticmethod + def connect(data_directory: Path) -> ibis.client.Client: + import dask.dataframe as dd + + # Note - we use `dd.from_pandas(pd.read_csv(...))` instead of + # `dd.read_csv` due to https://github.com/dask/dask/issues/6970 + + return ibis.backends.dask.connect( + { + 'functional_alltypes': dd.from_pandas( + pd.read_csv( + str(data_directory / 'functional_alltypes.csv'), + index_col=None, + dtype={'bool_col': bool, 'string_col': str}, + parse_dates=['timestamp_col'], + encoding='utf-8', + ), + npartitions=1, + ), + 'batting': dd.from_pandas( + pd.read_csv(str(data_directory / 'batting.csv')), + npartitions=1, + ), + 'awards_players': dd.from_pandas( + pd.read_csv(str(data_directory / 'awards_players.csv')), + npartitions=1, + ), + } + ) + + # @staticmethod + # def default_series_rename( + # series: pd.Series, name: str = 'tmp' + # ) -> pd.Series: + # return series.compute().rename(name) + + @classmethod + def assert_series_equal( + cls, left, right, *args: Any, **kwargs: Any + ) -> None: + import dask.dataframe as dd + + kwargs.setdefault('check_dtype', cls.check_dtype) + kwargs.setdefault('check_names', cls.check_names) + # we sometimes use pandas to build the "expected" case in tests + right = right.compute() if isinstance(right, dd.Series) else right + tm.assert_series_equal(left.compute(), right, *args, **kwargs) + + @classmethod + def assert_frame_equal( + cls, left, right, *args: Any, **kwargs: Any + ) -> None: + left = left.compute().reset_index(drop=True) + right = right.compute().reset_index(drop=True) + tm.assert_frame_equal(left, right, *args, **kwargs) class Csv(Pandas): diff --git a/setup.py b/setup.py index aa350cabe290..7475da584194 100644 --- a/setup.py +++ b/setup.py @@ -41,7 +41,7 @@ geospatial_requires = ['geoalchemy2', 'geopandas', 'shapely'] dask_requires = [ - 'dask[dataframe, array]', + 'dask[dataframe, array]>=2.22.0', ] all_requires = (