diff --git a/ibis/backends/bigquery/registry.py b/ibis/backends/bigquery/registry.py index 9c43ad2741ae..7911f9b87555 100644 --- a/ibis/backends/bigquery/registry.py +++ b/ibis/backends/bigquery/registry.py @@ -776,14 +776,62 @@ def _group_concat(translator, op): return f"STRING_AGG({arg}, {sep})" -def _integer_range(translator, op): - start = translator.translate(op.start) - stop = translator.translate(op.stop) - step = translator.translate(op.step) - n = f"FLOOR(({stop} - {start}) / NULLIF({step}, 0))" - gen_array = f"GENERATE_ARRAY({start}, {stop}, {step})" - inner = f"SELECT x FROM UNNEST({gen_array}) x WHERE x <> {stop}" - return f"IF({n} > 0, ARRAY({inner}), [])" +def _zero(dtype): + if dtype.is_interval(): + return "MAKE_INTERVAL()" + return "0" + + +def _sign(value, dtype): + if dtype.is_interval(): + zero = _zero(dtype) + return f"""\ +CASE + WHEN {value} < {zero} THEN -1 + WHEN {value} = {zero} THEN 0 + WHEN {value} > {zero} THEN 1 + ELSE NULL +END""" + return f"SIGN({value})" + + +def _nullifzero(step, zero, step_dtype): + if step_dtype.is_interval(): + return f"IF({step} = {zero}, NULL, {step})" + return f"NULLIF({step}, {zero})" + + +def _make_range(func): + def _range(translator, op): + start = translator.translate(op.start) + stop = translator.translate(op.stop) + step = translator.translate(op.step) + + step_dtype = op.step.dtype + step_sign = _sign(step, step_dtype) + delta_sign = _sign(step, step_dtype) + zero = _zero(step_dtype) + nullifzero = _nullifzero(step, zero, step_dtype) + + condition = f"{nullifzero} IS NOT NULL AND {step_sign} = {delta_sign}" + gen_array = f"{func}({start}, {stop}, {step})" + inner = f"SELECT x FROM UNNEST({gen_array}) x WHERE x <> {stop}" + return f"IF({condition}, ARRAY({inner}), [])" + + return _range + + +def _timestamp_range(translator, op): + start = op.start + stop = op.stop + + if start.dtype.timezone is None or stop.dtype.timezone is None: + raise com.IbisTypeError( + "Timestamps without timezone values are not supported when generating timestamp ranges" + ) + + rule = _make_range("GENERATE_TIMESTAMP_ARRAY") + return rule(translator, op) OPERATION_REGISTRY = { @@ -949,7 +997,8 @@ def _integer_range(translator, op): ops.TimeDelta: _time_delta, ops.DateDelta: _date_delta, ops.TimestampDelta: _timestamp_delta, - ops.IntegerRange: _integer_range, + ops.IntegerRange: _make_range("GENERATE_ARRAY"), + ops.TimestampRange: _timestamp_range, } _invalid_operations = { diff --git a/ibis/backends/clickhouse/__init__.py b/ibis/backends/clickhouse/__init__.py index 88fd3dd24c84..080c182879e2 100644 --- a/ibis/backends/clickhouse/__init__.py +++ b/ibis/backends/clickhouse/__init__.py @@ -3,6 +3,7 @@ import ast import atexit import glob +import warnings from contextlib import closing, suppress from functools import partial from typing import TYPE_CHECKING, Any, Literal @@ -169,6 +170,11 @@ def do_connect( compress=compression, **kwargs, ) + try: + with closing(self.raw_sql("SET session_timezone = 'UTC'")): + pass + except Exception as e: # noqa: BLE001 + warnings.warn(f"Could not set timezone to UTC: {e}", category=UserWarning) self._temp_views = set() @property diff --git a/ibis/backends/clickhouse/compiler/values.py b/ibis/backends/clickhouse/compiler/values.py index 404f1dc9f549..ba4163229135 100644 --- a/ibis/backends/clickhouse/compiler/values.py +++ b/ibis/backends/clickhouse/compiler/values.py @@ -1017,3 +1017,30 @@ def _agg_udf(op, *, where, **kw) -> str: @translate_val.register(ops.TimestampDelta) def _delta(op, *, part, left, right, **_): return sg.exp.DateDiff(this=left, expression=right, unit=part) + + +@translate_val.register(ops.TimestampRange) +def _timestamp_range(op, *, start, stop, step, **_): + unit = op.step.dtype.unit.name.lower() + + if not isinstance(op.step, ops.Literal): + raise com.UnsupportedOperationError( + "ClickHouse doesn't support non-literal step values" + ) + + step_value = op.step.value + + offset = sg.to_identifier("offset") + + # e.g., offset -> dateAdd(DAY, offset, start) + func = sg.exp.Lambda( + this=F.dateAdd(sg.to_identifier(unit), offset, start), expressions=[offset] + ) + + if step_value == 0: + return F.array() + + result = F.arrayMap( + func, F.range(0, F.timestampDiff(unit, start, stop), step_value) + ) + return result diff --git a/ibis/backends/duckdb/registry.py b/ibis/backends/duckdb/registry.py index e84ed3b07d54..f5a1489bbb91 100644 --- a/ibis/backends/duckdb/registry.py +++ b/ibis/backends/duckdb/registry.py @@ -183,7 +183,7 @@ def _literal(t, op): sqla_type = t.get_sqla_type(dtype) if dtype.is_interval(): - return sa.literal_column(f"INTERVAL '{value} {dtype.resolution}'") + return getattr(sa.func, f"to_{dtype.unit.plural}")(value) elif dtype.is_array(): values = value.tolist() if isinstance(value, np.ndarray) else value return sa.cast(sa.func.list_value(*values), sqla_type) @@ -550,6 +550,8 @@ def _array_remove(t, op): ops.GeoWithin: fixed_arity(sa.func.ST_Within, 2), ops.GeoX: unary(sa.func.ST_X), ops.GeoY: unary(sa.func.ST_Y), + # other ops + ops.TimestampRange: fixed_arity(sa.func.range, 3), } ) diff --git a/ibis/backends/polars/compiler.py b/ibis/backends/polars/compiler.py index d57165baa8fe..390a248dd81c 100644 --- a/ibis/backends/polars/compiler.py +++ b/ibis/backends/polars/compiler.py @@ -1205,7 +1205,9 @@ def execute_agg_udf(op, **kw): @translate.register(ops.IntegerRange) def execute_integer_range(op, **kw): if not isinstance(op.step, ops.Literal): - raise NotImplementedError("Dynamic step not supported by Polars") + raise com.UnsupportedOperationError( + "Dynamic integer step not supported by Polars" + ) step = op.step.value dtype = dtype_to_polars(op.dtype) @@ -1217,3 +1219,17 @@ def execute_integer_range(op, **kw): start = translate(op.start, **kw) stop = translate(op.stop, **kw) return pl.int_ranges(start, stop, step, dtype=dtype) + + +@translate.register(ops.TimestampRange) +def execute_timestamp_range(op, **kw): + if not isinstance(op.step, ops.Literal): + raise com.UnsupportedOperationError( + "Dynamic interval step not supported by Polars" + ) + step = op.step.value + unit = op.step.dtype.unit.value + + start = translate(op.start, **kw) + stop = translate(op.stop, **kw) + return pl.datetime_ranges(start, stop, f"{step}{unit}", closed="left") diff --git a/ibis/backends/postgres/registry.py b/ibis/backends/postgres/registry.py index 97761f2e0e76..a08c8546e438 100644 --- a/ibis/backends/postgres/registry.py +++ b/ibis/backends/postgres/registry.py @@ -618,19 +618,36 @@ def _array_filter(t, op): ) -def _integer_range(t, op): +def zero_value(dtype): + if dtype.is_interval(): + return sa.func.make_interval() + return 0 + + +def interval_sign(v): + zero = sa.func.make_interval() + return sa.case((v == zero, 0), (v < zero, -1), (v > zero, 1)) + + +def _sign(value, dtype): + if dtype.is_interval(): + return interval_sign(value) + return sa.func.sign(value) + + +def _range(t, op): start = t.translate(op.start) stop = t.translate(op.stop) step = t.translate(op.step) satype = t.get_sqla_type(op.dtype) - # `sequence` doesn't allow arguments that would produce an empty range, so - # check that first - n = sa.func.floor((stop - start) / sa.func.nullif(step, 0)) seq = sa.func.generate_series(start, stop, step, type_=satype) + zero = zero_value(op.step.dtype) return sa.case( - # TODO(cpcloud): revisit using array_remove when my brain is working ( - n > 0, + sa.and_( + sa.func.nullif(step, zero).is_not(None), + _sign(step, op.step.dtype) == _sign(stop - start, op.step.dtype), + ), sa.func.array_remove( sa.func.array(sa.select(seq).scalar_subquery()), stop, type_=satype ), @@ -839,6 +856,7 @@ def _integer_range(t, op): ops.ArrayPosition: fixed_arity(_array_position, 2), ops.ArrayMap: _array_map, ops.ArrayFilter: _array_filter, - ops.IntegerRange: _integer_range, + ops.IntegerRange: _range, + ops.TimestampRange: _range, } ) diff --git a/ibis/backends/pyspark/compiler.py b/ibis/backends/pyspark/compiler.py index ffe113f27b8a..190616b37aa9 100644 --- a/ibis/backends/pyspark/compiler.py +++ b/ibis/backends/pyspark/compiler.py @@ -2082,18 +2082,45 @@ def compile_flatten(t, op, **kwargs): return F.flatten(t.translate(op.arg, **kwargs)) +def _zero_value(dtype): + if dtype.is_interval(): + return F.expr(f"INTERVAL 0 {dtype.resolution}") + return F.lit(0) + + +def _build_sequence(start, stop, step, zero): + seq = F.sequence(start, stop, step) + length = F.size(seq) + last_element = F.element_at(seq, length) + # slice off the last element if we'd be inclusive on the right + seq = F.when(last_element == stop, F.slice(seq, 1, length - 1)).otherwise(seq) + return F.when( + (step != zero) & (F.signum(step) == F.signum(stop - start)), seq + ).otherwise(F.array()) + + @compiles(ops.IntegerRange) def compile_integer_range(t, op, **kwargs): start = t.translate(op.start, **kwargs) stop = t.translate(op.stop, **kwargs) step = t.translate(op.step, **kwargs) - denom = F.when(step == 0, F.lit(None)).otherwise(step) - n = F.floor((stop - start) / denom) - seq = F.sequence(start, stop, step) - seq = F.slice( - seq, - 1, - F.size(seq) - F.when(F.element_at(seq, F.size(seq)) == stop, 1).otherwise(0), - ) - return F.when(n > 0, seq).otherwise(F.array()) + return _build_sequence(start, stop, step, _zero_value(op.step.dtype)) + + +@compiles(ops.TimestampRange) +def compile_timestamp_range(t, op, **kwargs): + start = t.translate(op.start, **kwargs) + stop = t.translate(op.stop, **kwargs) + + if not isinstance(op.step, ops.Literal): + raise com.UnsupportedOperationError( + "`step` argument of timestamp range must be a literal" + ) + + step_value = op.step.value + unit = op.step.dtype.resolution + + step = F.expr(f"INTERVAL {step_value} {unit}") + + return _build_sequence(start, stop, step, _zero_value(op.step.dtype)) diff --git a/ibis/backends/snowflake/converter.py b/ibis/backends/snowflake/converter.py index 4d46ee53bbc3..4757401f021b 100644 --- a/ibis/backends/snowflake/converter.py +++ b/ibis/backends/snowflake/converter.py @@ -1,5 +1,6 @@ from __future__ import annotations +import datetime from typing import TYPE_CHECKING from ibis.formats.pandas import PandasData @@ -18,7 +19,31 @@ def convert_JSON(s, dtype, pandas_type): converter = SnowflakePandasData.convert_JSON_element(dtype) return s.map(converter, na_action="ignore").astype("object") - convert_Struct = convert_Array = convert_Map = convert_JSON + convert_Struct = convert_Map = convert_JSON + + @staticmethod + def get_element_converter(dtype): + funcgen = getattr( + SnowflakePandasData, + f"convert_{type(dtype).__name__}_element", + lambda _: lambda x: x, + ) + return funcgen(dtype) + + def convert_Timestamp_element(dtype): + return lambda values: list(map(datetime.datetime.fromisoformat, values)) + + def convert_Date_element(dtype): + return lambda values: list(map(datetime.date.fromisoformat, values)) + + def convert_Time_element(dtype): + return lambda values: list(map(datetime.time.fromisoformat, values)) + + @staticmethod + def convert_Array(s, dtype, pandas_type): + raw_json_objects = SnowflakePandasData.convert_JSON(s, dtype, pandas_type) + converter = SnowflakePandasData.get_element_converter(dtype.value_type) + return raw_json_objects.map(converter, na_action="ignore") class SnowflakePyArrowData(PyArrowData): diff --git a/ibis/backends/snowflake/registry.py b/ibis/backends/snowflake/registry.py index 31a9db4cbfa7..e31466ad64c1 100644 --- a/ibis/backends/snowflake/registry.py +++ b/ibis/backends/snowflake/registry.py @@ -280,6 +280,65 @@ def _timestamp_bucket(t, op): ) +class _flatten(sa.sql.functions.GenericFunction): + def __init__(self, arg, *, type: sa.types.TypeEngine) -> None: + super().__init__(arg) + self.type = sa.sql.sqltypes.TableValueType( + sa.Column("index", sa.BIGINT()), sa.Column("value", type) + ) + + +@compiles(_flatten, "snowflake") +def compiles_flatten(element, compiler, **kw): + (arg,) = element.clauses.clauses + return f"TABLE(FLATTEN(INPUT => {compiler.process(arg, **kw)}, MODE => 'ARRAY'))" + + +def _timestamp_range(t, op): + if not isinstance(op.step, ops.Literal): + raise com.UnsupportedOperationError("`step` argument must be a literal") + + start = t.translate(op.start) + stop = t.translate(op.stop) + + unit = op.step.dtype.unit.name.lower() + step = op.step.value + + value_type = op.dtype.value_type + + f = _flatten( + sa.func.array_generate_range(0, sa.func.datediff(unit, start, stop), step), + type=t.get_sqla_type(op.start.dtype), + ).alias("f") + return sa.func.iff( + step != 0, + sa.select( + sa.func.array_agg( + sa.func.replace( + # conversion to varchar is necessary to control + # the timestamp format + # + # otherwise, since timestamps in arrays become strings + # anyway due to lack of parameterized type support in + # Snowflake the format depends on a session parameter + sa.func.to_varchar( + sa.func.dateadd(unit, f.c.value, start), + 'YYYY-MM-DD"T"HH24:MI:SS.FF6' + + (value_type.timezone is not None) * "TZH:TZM", + ), + # timezones are always hour:minute offsets from UTC, not + # named, so replacing "Z" shouldn't be an issue + "Z", + "+00:00", + ), + ) + ) + .select_from(f) + .scalar_subquery(), + sa.func.array_construct(), + ) + + _TIMESTAMP_UNITS_TO_SCALE = {"s": 0, "ms": 3, "us": 6, "ns": 9} _SF_POS_INF = sa.func.to_double("Inf") @@ -504,6 +563,7 @@ def _timestamp_bucket(t, op): ), 3, ), + ops.TimestampRange: _timestamp_range, } ) diff --git a/ibis/backends/tests/test_array.py b/ibis/backends/tests/test_array.py index 10f319dab1c9..24c2b2028679 100644 --- a/ibis/backends/tests/test_array.py +++ b/ibis/backends/tests/test_array.py @@ -2,11 +2,13 @@ import contextlib import functools +from datetime import datetime import numpy as np import pandas as pd import pandas.testing as tm import pytest +import pytz import sqlalchemy as sa import toolz from pytest import param @@ -34,6 +36,17 @@ except ImportError: PySparkAnalysisException = None +try: + from polars.exceptions import ComputeError as PolarsComputeError +except ImportError: + PolarsComputeError = None + +try: + from py4j.protocol import Py4JJavaError +except ImportError: + Py4JJavaError = None + + pytestmark = [ pytest.mark.never( ["sqlite", "mysql", "mssql", "exasol"], @@ -931,3 +944,134 @@ def swap(token): expr = arr.map(lambda token: token.substitute({"abc": "ABC"})) assert con.execute(expr) == ["ABC", "xyz"] + + +timestamp_range_tzinfos = pytest.mark.parametrize( + "tzinfo", + [ + param( + None, + id="none", + marks=[ + pytest.mark.notyet( + ["bigquery"], + raises=com.IbisTypeError, + reason="bigquery doesn't support datetime ranges, only timestamp ranges", + ), + ], + ), + param( + pytz.UTC, + id="utc", + marks=[ + pytest.mark.notyet( + ["trino"], + raises=sa.exc.ProgrammingError, + reason="trino doesn't support timestamp with time zone arguments to its sequence function", + ), + pytest.mark.notyet( + ["polars"], + raises=(TypeError, com.UnsupportedOperationError), + reason="polars doesn't work with dateutil timezones", + ), + ], + ), + ], +) + + +@pytest.mark.parametrize( + ("start", "stop", "step", "freq"), + [ + param( + datetime(2017, 1, 1), + datetime(2017, 1, 2), + ibis.interval(hours=1), + "1H", + id="pos", + ), + param( + datetime(2017, 1, 2), + datetime(2017, 1, 1), + ibis.interval(hours=-1), + "-1H", + id="neg_inner", + marks=[pytest.mark.notyet(["polars"], raises=PolarsComputeError)], + ), + param( + datetime(2017, 1, 2), + datetime(2017, 1, 1), + -ibis.interval(hours=1), + "-1H", + id="neg_outer", + marks=[ + pytest.mark.notyet(["polars"], raises=com.UnsupportedOperationError), + pytest.mark.notyet(["bigquery"], raises=BadRequest), + pytest.mark.notyet( + ["clickhouse", "pyspark", "snowflake"], + raises=com.UnsupportedOperationError, + ), + ], + ), + ], +) +@timestamp_range_tzinfos +@pytest.mark.notimpl( + ["pandas", "dask", "flink", "datafusion"], raises=com.OperationNotDefinedError +) +def test_timestamp_range(con, start, stop, step, freq, tzinfo): + start = start.replace(tzinfo=tzinfo) + stop = stop.replace(tzinfo=tzinfo) + expr = ibis.range(start, stop, step) + result = con.execute(expr) + expected = pd.date_range(start, stop, freq=freq, inclusive="left") + assert list(result) == expected.tolist() + + +@pytest.mark.parametrize( + ("start", "stop", "step"), + [ + param( + datetime(2017, 1, 1, tzinfo=pytz.UTC), + datetime(2017, 1, 2, tzinfo=pytz.UTC), + ibis.interval(hours=0), + id="pos", + marks=[pytest.mark.notyet(["polars"], raises=PolarsComputeError)], + ), + param( + datetime(2017, 1, 1, tzinfo=pytz.UTC), + datetime(2017, 1, 2, tzinfo=pytz.UTC), + -ibis.interval(hours=0), + id="neg", + marks=[ + pytest.mark.notyet(["polars"], raises=com.UnsupportedOperationError), + pytest.mark.notyet(["bigquery"], raises=BadRequest), + pytest.mark.notyet( + ["clickhouse", "pyspark", "snowflake"], + raises=com.UnsupportedOperationError, + ), + ], + ), + ], +) +@timestamp_range_tzinfos +@pytest.mark.notimpl( + ["pandas", "dask", "flink", "datafusion"], raises=com.OperationNotDefinedError +) +def test_timestamp_range_zero_step(con, start, stop, step, tzinfo): + start = start.replace(tzinfo=tzinfo) + stop = stop.replace(tzinfo=tzinfo) + expr = ibis.range(start, stop, step) + result = con.execute(expr) + assert list(result) == [] + + +@pytest.mark.notimpl(["flink"], raises=Py4JJavaError) +@pytest.mark.notimpl(["datafusion"], raises=Exception) +def test_repr_timestamp_array(con, monkeypatch): + monkeypatch.setattr(ibis.options, "interactive", True) + monkeypatch.setattr(ibis.options, "default_backend", con) + assert ibis.options.interactive is True + assert ibis.options.default_backend is con + expr = ibis.array(pd.date_range("2010-01-01", "2010-01-03", freq="D").tolist()) + assert repr(expr) diff --git a/ibis/backends/trino/registry.py b/ibis/backends/trino/registry.py index a100a034a022..ab1427daccd0 100644 --- a/ibis/backends/trino/registry.py +++ b/ibis/backends/trino/registry.py @@ -350,17 +350,35 @@ def _interval_from_integer(t, op): return sa.type_coerce(sa.func.parse_duration(arg), INTERVAL) -def _integer_range(t, op): +def zero_value(dtype): + if dtype.is_interval(): + # the unit doesn't matter here, because e.g. 0d = 0s + return sa.func.parse_duration("0s") + return 0 + + +def interval_sign(v): + zero = sa.func.parse_duration("0s") + return sa.case((v == zero, 0), (v < zero, -1), (v > zero, 1)) + + +def _sign(value, dtype): + if dtype.is_interval(): + return interval_sign(value) + return sa.func.sign(value) + + +def _range(t, op): start = t.translate(op.start) stop = t.translate(op.stop) step = t.translate(op.step) satype = t.get_sqla_type(op.dtype) - # `sequence` doesn't allow arguments that would produce an empty range, so - # check that first - n = sa.func.floor((stop - start) / sa.func.nullif(step, 0)) + zero = zero_value(op.step.dtype) return if_( - n > 0, - # TODO(cpcloud): revisit using array_remove when my brain is working + sa.and_( + sa.func.nullif(step, zero).is_not(None), + _sign(step, op.step.dtype) == _sign(stop - start, op.step.dtype), + ), sa.func.array_remove( sa.func.sequence(start, stop, step, type_=satype), stop, type_=satype ), @@ -565,7 +583,8 @@ def _integer_range(t, op): ops.IntervalAdd: fixed_arity(operator.add, 2), ops.IntervalSubtract: fixed_arity(operator.sub, 2), ops.IntervalFromInteger: _interval_from_integer, - ops.IntegerRange: _integer_range, + ops.IntegerRange: _range, + ops.TimestampRange: _range, } ) diff --git a/ibis/expr/api.py b/ibis/expr/api.py index 9d1ddc31cf17..9f3440f446f1 100644 --- a/ibis/expr/api.py +++ b/ibis/expr/api.py @@ -1977,6 +1977,8 @@ def watermark(time_col: str, allowed_delay: ir.IntervalScalar) -> Watermark: def range(start, stop, step) -> ir.ArrayValue: """Generate a range of values. + Integer ranges are supported, as well as timestamp ranges. + ::: {.callout-note} `start` is inclucive and `stop` is exclusive, just like Python's builtin [`range`](range). @@ -2008,12 +2010,12 @@ def range(start, stop, step) -> ir.ArrayValue: Range using only a stop argument >>> ibis.range(5) - [0, 1, 2, 3, 4] + [0, 1, ... +3] Simple range using start and stop >>> ibis.range(1, 5) - [1, 2, 3, 4] + [1, 2, ... +2] Generate an empty range @@ -2023,7 +2025,7 @@ def range(start, stop, step) -> ir.ArrayValue: Negative step values are supported >>> ibis.range(10, 4, -2) - [10, 8, 6] + [10, 8, ... +1] `ibis.range` behaves the same as Python's range ... @@ -2050,6 +2052,34 @@ def range(start, stop, step) -> ir.ArrayValue: │ 3 │ │ 4 │ └─────────┘ + + Timestamp ranges are also supported + + >>> expr = ibis.range("2002-01-01", "2002-02-01", ibis.interval(days=2)).name("ts") + >>> expr + [ + datetime.datetime(2002, 1, 1, 0, 0), + datetime.datetime(2002, 1, 3, 0, 0), + ... +14 + ] + >>> expr.unnest() + ┏━━━━━━━━━━━━━━━━━━━━━┓ + ┃ ts ┃ + ┡━━━━━━━━━━━━━━━━━━━━━┩ + │ timestamp │ + ├─────────────────────┤ + │ 2002-01-01 00:00:00 │ + │ 2002-01-03 00:00:00 │ + │ 2002-01-05 00:00:00 │ + │ 2002-01-07 00:00:00 │ + │ 2002-01-09 00:00:00 │ + │ 2002-01-11 00:00:00 │ + │ 2002-01-13 00:00:00 │ + │ 2002-01-15 00:00:00 │ + │ 2002-01-17 00:00:00 │ + │ 2002-01-19 00:00:00 │ + │ … │ + └─────────────────────┘ """ raise NotImplementedError() @@ -2069,6 +2099,19 @@ def _int_range( return ops.IntegerRange(start=start, stop=stop, step=step).to_expr() +@range.register(str) +@range.register(datetime.datetime) +@range.register(ir.TimestampValue) +def _timestamp_range( + start: datetime.datetime | ir.TimestampValue | str, + stop: datetime.datetime | ir.TimestampValue | str, + step: datetime.timedelta | ir.IntervalValue, +) -> ir.ArrayValue: + return ops.TimestampRange( + start=normalize_datetime(start), stop=normalize_datetime(stop), step=step + ).to_expr() + + def _wrap_deprecated(fn, prefix=""): """Deprecate the top-level geo function.""" diff --git a/ibis/expr/operations/arrays.py b/ibis/expr/operations/arrays.py index 8b00ae5a17b6..c0e65b36272e 100644 --- a/ibis/expr/operations/arrays.py +++ b/ibis/expr/operations/arrays.py @@ -217,3 +217,10 @@ class IntegerRange(Range): start: Value[dt.Integer] stop: Value[dt.Integer] step: Value[dt.Integer] + + +@public +class TimestampRange(Range): + start: Value[dt.Timestamp] + stop: Value[dt.Timestamp] + step: Value[dt.Interval] diff --git a/ibis/expr/types/arrays.py b/ibis/expr/types/arrays.py index 2ba788df00e7..8258b4278ecf 100644 --- a/ibis/expr/types/arrays.py +++ b/ibis/expr/types/arrays.py @@ -1070,7 +1070,7 @@ def array(values: Iterable[V], type: str | dt.DataType | None = None) -> ArrayVa Create an array scalar from Python literals >>> ibis.array([1.0, 2.0, 3.0]) - [1.0, 2.0, 3.0] + [1.0, 2.0, ... +1] Mixing scalar and column expressions is allowed diff --git a/ibis/expr/types/generic.py b/ibis/expr/types/generic.py index 8a9758834056..f41ccc686216 100644 --- a/ibis/expr/types/generic.py +++ b/ibis/expr/types/generic.py @@ -1027,7 +1027,7 @@ def collect(self, where: ir.BooleanValue | None = None) -> ir.ArrayScalar: │ b │ 5 │ └────────┴───────┘ >>> t.value.collect() - [1, 2, 3, 4, 5] + [1, 2, ... +3] >>> type(t.value.collect()) @@ -1229,7 +1229,18 @@ def to_pandas(self, **kwargs) -> pd.Series: @public class Scalar(Value): def __interactive_rich_console__(self, console, options): - return console.render(repr(self.execute()), options=options) + import rich.pretty + + interactive = ibis.options.repr.interactive + return console.render( + rich.pretty.Pretty( + self.execute(), + max_length=interactive.max_length, + max_string=interactive.max_string, + max_depth=interactive.max_depth, + ), + options=options, + ) def __pyarrow_result__( self, table: pa.Table, data_mapper: type[PyArrowData] | None = None diff --git a/ibis/expr/types/maps.py b/ibis/expr/types/maps.py index 4de29aeba6b4..9ef4b88c2a26 100644 --- a/ibis/expr/types/maps.py +++ b/ibis/expr/types/maps.py @@ -385,7 +385,7 @@ def __add__(self, other: MapValue) -> MapValue: >>> m1 = ibis.map({"a": 1, "b": 2}) >>> m2 = ibis.map({"c": 3, "d": 4}) >>> m1 + m2 - {'a': 1, 'b': 2, 'c': 3, 'd': 4} + {'a': 1, 'b': 2, ... +2} """ return ops.MapMerge(self, other).to_expr() @@ -409,7 +409,7 @@ def __radd__(self, other: MapValue) -> MapValue: >>> m1 = ibis.map({"a": 1, "b": 2}) >>> m2 = ibis.map({"c": 3, "d": 4}) >>> m1 + m2 - {'a': 1, 'b': 2, 'c': 3, 'd': 4} + {'a': 1, 'b': 2, ... +2} """ return ops.MapMerge(self, other).to_expr()