From a2be8a362907853e5ffc0f9f7c49326c99df9fda Mon Sep 17 00:00:00 2001 From: Andrew Gazelka Date: Mon, 30 Sep 2024 20:32:12 -0700 Subject: [PATCH 1/5] [CHORE] ignore 45e2944e (#2979) --- .git-blame-ignore-revs | 1 + 1 file changed, 1 insertion(+) diff --git a/.git-blame-ignore-revs b/.git-blame-ignore-revs index ef7507bc66..c017f4cf5b 100644 --- a/.git-blame-ignore-revs +++ b/.git-blame-ignore-revs @@ -1 +1,2 @@ d5e444d0a71409ae3701d4249ad877f1fb9e2235 # introduced `rustfmt.toml` and ran formatter; ignoring large formatting changes +45e2944e252ccdd563dc20edd9b29762e05cec1d # auto-fix prefer `Self` over explicit type From f4d1da2d00d92b8bcc154c2df1b3e47384839492 Mon Sep 17 00:00:00 2001 From: Jay Chia <17691182+jaychia@users.noreply.github.com> Date: Mon, 30 Sep 2024 22:31:23 -0700 Subject: [PATCH 2/5] [DOCS] Fix docs to add SQL capabilities (#2931) 1. Changes a lot of the messaging in Daft to talk about us as a **data engine** that exposes both a **SQL and Python API** --------- Co-authored-by: Jay Chia --- daft/daft/__init__.pyi | 10 +- daft/sql/_sql_funcs.py | 30 + daft/sql/sql.py | 118 +++- docs/source/10-min.ipynb | 4 +- docs/source/api_docs/index.rst | 1 + docs/source/api_docs/sql.rst | 15 + docs/source/conf.py | 16 + docs/source/ext/__init__.py | 0 docs/source/ext/sql_autosummary.py | 80 +++ docs/source/index.rst | 55 +- .../migration_guides/coming_from_dask.rst | 6 +- .../{daft_in_depth => }/aggregations.rst | 0 docs/source/user_guide/basic_concepts.rst | 408 +++++++++++- .../basic_concepts/dataframe_introduction.rst | 203 ------ .../user_guide/basic_concepts/expressions.rst | 343 ---------- .../basic_concepts/introduction.rst | 92 --- docs/source/user_guide/daft_in_depth.rst | 9 - .../dataframe-operations.rst | 0 .../{daft_in_depth => }/datatypes.rst | 0 docs/source/user_guide/expressions.rst | 584 ++++++++++++++++++ .../user_guide/fotw/fotw-001-images.ipynb | 4 +- docs/source/user_guide/index.rst | 44 +- .../{basic_concepts => }/read-and-write.rst | 51 +- docs/source/user_guide/sql.rst | 244 ++++++++ .../user_guide/{daft_in_depth => }/udf.rst | 0 requirements-dev.txt | 1 + src/daft-sql/src/functions.rs | 14 + src/daft-sql/src/modules/aggs.rs | 218 +++++++ src/daft-sql/src/modules/float.rs | 46 ++ src/daft-sql/src/modules/image/crop.rs | 8 + src/daft-sql/src/modules/image/decode.rs | 8 + src/daft-sql/src/modules/image/encode.rs | 8 + src/daft-sql/src/modules/image/resize.rs | 8 + src/daft-sql/src/modules/image/to_mode.rs | 8 + src/daft-sql/src/modules/json.rs | 13 + src/daft-sql/src/modules/list.rs | 106 ++++ src/daft-sql/src/modules/map.rs | 19 + src/daft-sql/src/modules/numeric.rs | 61 ++ src/daft-sql/src/modules/partitioning.rs | 22 + src/daft-sql/src/modules/structs.rs | 8 + src/daft-sql/src/modules/temporal.rs | 10 + src/daft-sql/src/modules/utf8.rs | 66 ++ src/daft-sql/src/python.rs | 41 +- 43 files changed, 2277 insertions(+), 705 deletions(-) create mode 100644 daft/sql/_sql_funcs.py create mode 100644 docs/source/api_docs/sql.rst create mode 100644 docs/source/ext/__init__.py create mode 100644 docs/source/ext/sql_autosummary.py rename docs/source/user_guide/{daft_in_depth => }/aggregations.rst (100%) delete mode 100644 docs/source/user_guide/basic_concepts/dataframe_introduction.rst delete mode 100644 docs/source/user_guide/basic_concepts/expressions.rst delete mode 100644 docs/source/user_guide/basic_concepts/introduction.rst delete mode 100644 docs/source/user_guide/daft_in_depth.rst rename docs/source/user_guide/{daft_in_depth => }/dataframe-operations.rst (100%) rename docs/source/user_guide/{daft_in_depth => }/datatypes.rst (100%) create mode 100644 docs/source/user_guide/expressions.rst rename docs/source/user_guide/{basic_concepts => }/read-and-write.rst (64%) create mode 100644 docs/source/user_guide/sql.rst rename docs/source/user_guide/{daft_in_depth => }/udf.rst (100%) diff --git a/daft/daft/__init__.pyi b/daft/daft/__init__.pyi index 08ec0860ba..c90817dfc2 100644 --- a/daft/daft/__init__.pyi +++ b/daft/daft/__init__.pyi @@ -1201,9 +1201,17 @@ def minhash( # ----- # SQL functions # ----- +class SQLFunctionStub: + @property + def name(self) -> str: ... + @property + def docstring(self) -> str: ... + @property + def arg_names(self) -> list[str]: ... + def sql(sql: str, catalog: PyCatalog, daft_planning_config: PyDaftPlanningConfig) -> LogicalPlanBuilder: ... def sql_expr(sql: str) -> PyExpr: ... -def list_sql_functions() -> list[str]: ... +def list_sql_functions() -> list[SQLFunctionStub]: ... def utf8_count_matches(expr: PyExpr, patterns: PyExpr, whole_words: bool, case_sensitive: bool) -> PyExpr: ... def to_struct(inputs: list[PyExpr]) -> PyExpr: ... diff --git a/daft/sql/_sql_funcs.py b/daft/sql/_sql_funcs.py new file mode 100644 index 0000000000..030cd3b53f --- /dev/null +++ b/daft/sql/_sql_funcs.py @@ -0,0 +1,30 @@ +"""This module is used for Sphinx documentation only. We procedurally generate Python functions to allow +Sphinx to generate documentation pages for every SQL function. +""" + +from __future__ import annotations + +from inspect import Parameter as _Parameter +from inspect import Signature as _Signature + +from daft.daft import list_sql_functions as _list_sql_functions + + +def _create_sql_function(func_name: str, docstring: str, arg_names: list[str]): + def sql_function(*args, **kwargs): + raise NotImplementedError("This function is for documentation purposes only and should not be called.") + + sql_function.__name__ = func_name + sql_function.__qualname__ = func_name + sql_function.__doc__ = docstring + sql_function.__signature__ = _Signature([_Parameter(name, _Parameter.POSITIONAL_OR_KEYWORD) for name in arg_names]) # type: ignore[attr-defined] + + # Register the function in the current module + globals()[func_name] = sql_function + + +__all__ = [] + +for sql_function_stub in _list_sql_functions(): + _create_sql_function(sql_function_stub.name, sql_function_stub.docstring, sql_function_stub.arg_names) + __all__.append(sql_function_stub.name) diff --git a/daft/sql/sql.py b/daft/sql/sql.py index 987a9baeb0..2c9bb78554 100644 --- a/daft/sql/sql.py +++ b/daft/sql/sql.py @@ -1,7 +1,7 @@ # isort: dont-add-import: from __future__ import annotations import inspect -from typing import Optional, overload +from typing import Optional from daft.api_annotations import PublicAPI from daft.context import get_context @@ -38,22 +38,120 @@ def _copy_from(self, other: "SQLCatalog") -> None: @PublicAPI def sql_expr(sql: str) -> Expression: - return Expression._from_pyexpr(_sql_expr(sql)) - + """Parses a SQL string into a Daft Expression -@overload -def sql(sql: str) -> DataFrame: ... + This function allows you to create Daft Expressions from SQL snippets, which can then be used + in Daft operations or combined with other Daft Expressions. + Args: + sql (str): A SQL string to be parsed into a Daft Expression. -@overload -def sql(sql: str, catalog: SQLCatalog, register_globals: bool = ...) -> DataFrame: ... + Returns: + Expression: A Daft Expression representing the parsed SQL. + + Examples: + Create a simple SQL expression: + + >>> import daft + >>> expr = daft.sql_expr("1 + 2") + >>> print(expr) + lit(1) + lit(2) + + Use SQL expression in a Daft DataFrame operation: + + >>> df = daft.from_pydict({"a": [1, 2, 3], "b": [4, 5, 6]}) + >>> df = df.with_column("c", daft.sql_expr("a + b")) + >>> df.show() + ╭───────┬───────┬───────╮ + │ a ┆ b ┆ c │ + │ --- ┆ --- ┆ --- │ + │ Int64 ┆ Int64 ┆ Int64 │ + ╞═══════╪═══════╪═══════╡ + │ 1 ┆ 4 ┆ 5 │ + ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌┤ + │ 2 ┆ 5 ┆ 7 │ + ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌┤ + │ 3 ┆ 6 ┆ 9 │ + ╰───────┴───────┴───────╯ + + (Showing first 3 of 3 rows) + + `daft.sql_expr` is also called automatically for you in some DataFrame operations such as filters: + + >>> df = daft.from_pydict({"x": [1, 2, 3], "y": [4, 5, 6]}) + >>> result = df.where("x < 3 AND y > 4") + >>> result.show() + ╭───────┬───────╮ + │ x ┆ y │ + │ --- ┆ --- │ + │ Int64 ┆ Int64 │ + ╞═══════╪═══════╡ + │ 2 ┆ 5 │ + ╰───────┴───────╯ + + (Showing first 1 of 1 rows) + """ + return Expression._from_pyexpr(_sql_expr(sql)) @PublicAPI def sql(sql: str, catalog: Optional[SQLCatalog] = None, register_globals: bool = True) -> DataFrame: - """Create a DataFrame from an SQL query. - - EXPERIMENTAL: This features is early in development and will change. + """Run a SQL query, returning the results as a DataFrame + + .. WARNING:: + This features is early in development and will likely experience API changes. + + Examples: + + A simple example joining 2 dataframes together using a SQL statement, relying on Daft to detect the names of + SQL tables using their corresponding Python variable names. + + >>> import daft + >>> + >>> df1 = daft.from_pydict({"a": [1, 2, 3], "b": ["foo", "bar", "baz"]}) + >>> df2 = daft.from_pydict({"a": [1, 2, 3], "c": ["daft", None, None]}) + >>> + >>> # Daft automatically detects `df1` and `df2` from your Python global namespace + >>> result_df = daft.sql("SELECT * FROM df1 JOIN df2 ON df1.a = df2.a") + >>> result_df.show() + ╭───────┬──────┬──────╮ + │ a ┆ b ┆ c │ + │ --- ┆ --- ┆ --- │ + │ Int64 ┆ Utf8 ┆ Utf8 │ + ╞═══════╪══════╪══════╡ + │ 1 ┆ foo ┆ daft │ + ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌┼╌╌╌╌╌╌┤ + │ 2 ┆ bar ┆ None │ + ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌┼╌╌╌╌╌╌┤ + │ 3 ┆ baz ┆ None │ + ╰───────┴──────┴──────╯ + + (Showing first 3 of 3 rows) + + A more complex example using a SQLCatalog to create a named table called `"my_table"`, which can then be referenced from inside your SQL statement. + + >>> import daft + >>> from daft.sql import SQLCatalog + >>> + >>> df = daft.from_pydict({"a": [1, 2, 3], "b": ["foo", "bar", "baz"]}) + >>> + >>> # Register dataframes as tables in SQL explicitly with names + >>> catalog = SQLCatalog({"my_table": df}) + >>> + >>> daft.sql("SELECT a FROM my_table", catalog=catalog).show() + ╭───────╮ + │ a │ + │ --- │ + │ Int64 │ + ╞═══════╡ + │ 1 │ + ├╌╌╌╌╌╌╌┤ + │ 2 │ + ├╌╌╌╌╌╌╌┤ + │ 3 │ + ╰───────╯ + + (Showing first 3 of 3 rows) Args: sql (str): SQL query to execute diff --git a/docs/source/10-min.ipynb b/docs/source/10-min.ipynb index cbda803752..d4444c2cd5 100644 --- a/docs/source/10-min.ipynb +++ b/docs/source/10-min.ipynb @@ -569,7 +569,7 @@ "cell_type": "markdown", "metadata": {}, "source": [ - "See: [Expressions](user_guide/basic_concepts/expressions.rst)\n", + "See: [Expressions](user_guide/expressions.rst)\n", "\n", "Expressions are an API for defining computation that needs to happen over your columns.\n", "\n", @@ -1516,7 +1516,7 @@ "source": [ "### User-Defined Functions\n", "\n", - "See: [UDF User Guide](user_guide/daft_in_depth/udf)" + "See: [UDF User Guide](user_guide/udf)" ] }, { diff --git a/docs/source/api_docs/index.rst b/docs/source/api_docs/index.rst index 3079870df6..6bee44ad95 100644 --- a/docs/source/api_docs/index.rst +++ b/docs/source/api_docs/index.rst @@ -7,6 +7,7 @@ API Documentation Table of Contents creation dataframe + sql expressions schema datatype diff --git a/docs/source/api_docs/sql.rst b/docs/source/api_docs/sql.rst new file mode 100644 index 0000000000..33cf0c25dd --- /dev/null +++ b/docs/source/api_docs/sql.rst @@ -0,0 +1,15 @@ +SQL +=== + +.. autofunction:: daft.sql + +.. autofunction:: daft.sql_expr + +SQL Functions +------------- + +This is a full list of functions that can be used from within SQL. + + +.. sql-autosummary:: + :toctree: doc_gen/sql_funcs diff --git a/docs/source/conf.py b/docs/source/conf.py index 36e66be49a..fd59d32625 100644 --- a/docs/source/conf.py +++ b/docs/source/conf.py @@ -9,12 +9,16 @@ import inspect import os import subprocess +import sys import sphinx_autosummary_accessors # Set environment variable to help code determine whether or not we are running a Sphinx doc build process os.environ["DAFT_SPHINX_BUILD"] = "1" +# Help Sphinx find local custom extensions/directives that we build +sys.path.insert(0, os.path.abspath("ext")) + # -- Project information ----------------------------------------------------- # https://www.sphinx-doc.org/en/master/usage/configuration.html#project-information project = "Daft" @@ -45,10 +49,15 @@ "myst_nb", "sphinx_copybutton", "sphinx_autosummary_accessors", + "sphinx_tabs.tabs", + # Local extensions + "sql_autosummary", ] templates_path = ["_templates", sphinx_autosummary_accessors.templates_path] +# Removes module names that prefix our classes +add_module_names = False # -- Options for Notebook rendering # https://myst-nb.readthedocs.io/en/latest/configuration.html?highlight=nb_execution_mode#execution @@ -86,6 +95,13 @@ "learn/user_guides/remote_cluster_execution": "distributed-computing.html", "learn/quickstart": "learn/10-min.html", "learn/10-min": "../10-min.html", + "user_guide/basic_concepts/expressions": "user_guide/expressions", + "user_guide/basic_concepts/dataframe_introduction": "user_guide/basic_concepts", + "user_guide/basic_concepts/introduction": "user_guide/basic_concepts", + "user_guide/daft_in_depth/aggregations": "user_guide/aggregations", + "user_guide/daft_in_depth/dataframe-operations": "user_guide/dataframe-operations", + "user_guide/daft_in_depth/datatypes": "user_guide/datatypes", + "user_guide/daft_in_depth/udf": "user_guide/udf", } # Resolving code links to github diff --git a/docs/source/ext/__init__.py b/docs/source/ext/__init__.py new file mode 100644 index 0000000000..e69de29bb2 diff --git a/docs/source/ext/sql_autosummary.py b/docs/source/ext/sql_autosummary.py new file mode 100644 index 0000000000..5e37456cbe --- /dev/null +++ b/docs/source/ext/sql_autosummary.py @@ -0,0 +1,80 @@ +import inspect +import os + +from sphinx.ext.autosummary import Autosummary +from sphinx.util import logging + +logger = logging.getLogger(__name__) + + +TOCTREE = "doc_gen/sql_funcs" +SQL_MODULE_NAME = "daft.sql._sql_funcs" + +STUB_TEMPLATE = """ +.. currentmodule:: None + +.. autofunction:: {module_name}.{name} +""" + + +class SQLAutosummary(Autosummary): + def run(self): + func_names = get_sql_func_names() + # Run the normal autosummary stuff, override self.content + self.content = [f"~{SQL_MODULE_NAME}.{f}" for f in func_names] + nodes = super().run() + return nodes + + def get_sql_module_name(self): + return self.arguments[0] + + +def get_sql_func_names(): + # Import the SQL functions module + module = __import__(SQL_MODULE_NAME, fromlist=[""]) + + names = [] + for name, obj in inspect.getmembers(module): + if inspect.isfunction(obj) and not name.startswith("_"): + names.append(name) + + return names + + +def generate_stub(name: str): + """Generates a stub string for a SQL function""" + stub = name + "\n" + stub += "=" * len(name) + "\n\n" + stub += STUB_TEMPLATE.format(module_name=SQL_MODULE_NAME, name=name) + return stub + + +def generate_files(app): + # Determine where to write .rst files to + output_dir = os.path.join(app.srcdir, "api_docs", TOCTREE) + os.makedirs(output_dir, exist_ok=True) + + # Write stubfiles + func_names = get_sql_func_names() + for name in func_names: + stub_content = generate_stub(name) + filename = f"{SQL_MODULE_NAME}.{name}.rst" + filepath = os.path.join(output_dir, filename) + with open(filepath, "w") as f: + f.write(stub_content) + + # HACK: Not sure if this is ok? + app.env.found_docs.add(filepath) + + +def setup(app): + app.add_directive("sql-autosummary", SQLAutosummary) + + # Generate and register files when the builder is initialized + app.connect("builder-inited", generate_files) + + return { + "version": "0.1", + "parallel_read_safe": True, + "parallel_write_safe": True, + } diff --git a/docs/source/index.rst b/docs/source/index.rst index 3a2d3eabb5..6ee5c431b7 100644 --- a/docs/source/index.rst +++ b/docs/source/index.rst @@ -1,14 +1,53 @@ Daft Documentation ================== -Daft is a distributed query engine for large-scale data processing in Python and is implemented in Rust. - -* **Familiar interactive API:** Lazy Python Dataframe for rapid and interactive iteration -* **Focus on the what:** Powerful Query Optimizer that rewrites queries to be as efficient as possible -* **Data Catalog integrations:** Full integration with data catalogs such as Apache Iceberg -* **Rich multimodal type-system:** Supports multimodal types such as Images, URLs, Tensors and more -* **Seamless Interchange**: Built on the `Apache Arrow `_ In-Memory Format -* **Built for the cloud:** `Record-setting `_ I/O performance for integrations with S3 cloud storage +Daft is a unified data engine for **data engineering, analytics and ML/AI**. + +Daft exposes both **SQL and Python DataFrame interfaces** as first-class citizens and is written in Rust. + +Daft provides a **snappy and delightful local interactive experience**, but also seamlessly **scales to petabyte-scale distributed workloads**. + +Use-Cases +--------- + +Data Engineering +**************** + +*Combine the performance of DuckDB, Pythonic UX of Polars and scalability of Apache Spark for data engineering from MB to PB scale* + +* Scale ETL workflows effortlessly from local to distributed environments +* Enjoy a Python-first experience without JVM dependency hell +* Leverage native integrations with cloud storage, open catalogs, and data formats + +Data Analytics +************** + +*Blend the snappiness of DuckDB with the scalability of Spark/Trino for unified local and distributed analytics* + +* Utilize complementary SQL and Python interfaces for versatile analytics +* Perform snappy local exploration with DuckDB-like performance +* Seamlessly scale to the cloud, outperforming distributed engines like Spark and Trino + +ML/AI +***** + +*Streamline ML/AI workflows with efficient dataloading from open formats like Parquet and JPEG* + +* Load data efficiently from open formats directly into PyTorch or NumPy +* Schedule large-scale model batch inference on distributed GPU clusters +* Optimize data curation with advanced clustering, deduplication, and filtering + +Technology +---------- + +Daft boasts strong integrations with technologies common across these workloads: + +* **Cloud Object Storage:** Record-setting I/O performance for integrations with S3 cloud storage, `battle-tested at exabyte-scale at Amazon `_ +* **ML/AI Python Ecosystem:** first-class integrations with `PyTorch `_ and `NumPy `_ for efficient interoperability with your ML/AI stack +* **Data Catalogs/Table Formats:** capabilities to effectively query table formats such as `Apache Iceberg `_, `Delta Lake `_ and `Apache Hudi `_ +* **Seamless Data Interchange:** zero-copy integration with `Apache Arrow `_ +* **Multimodal/ML Data:** native functionality for data modalities such as tensors, images, URLs, long-form text and embeddings + Installing Daft --------------- diff --git a/docs/source/migration_guides/coming_from_dask.rst b/docs/source/migration_guides/coming_from_dask.rst index 4e649ec8d3..99606c3ff9 100644 --- a/docs/source/migration_guides/coming_from_dask.rst +++ b/docs/source/migration_guides/coming_from_dask.rst @@ -30,7 +30,7 @@ Daft does not use an index Dask aims for as much feature-parity with pandas as possible, including maintaining the presence of an Index in the DataFrame. But keeping an Index is difficult when moving to a distributed computing environment. Dask doesn’t support row-based positional indexing (with .iloc) because it does not track the length of its partitions. It also does not support pandas MultiIndex. The argument for keeping the Index is that it makes some operations against the sorted index column very fast. In reality, resetting the Index forces a data shuffle and is an expensive operation. -Daft drops the need for an Index to make queries more readable and consistent. How you write a query should not change because of the state of an index or a reset_index call. In our opinion, eliminating the index makes things simpler, more explicit, more readable and therefore less error-prone. Daft achieves this by using the [Expressions API](../user_guide/basic_concepts/expressions). +Daft drops the need for an Index to make queries more readable and consistent. How you write a query should not change because of the state of an index or a reset_index call. In our opinion, eliminating the index makes things simpler, more explicit, more readable and therefore less error-prone. Daft achieves this by using the [Expressions API](../user_guide/expressions). In Dask you would index your DataFrame to return row ``b`` as follows: @@ -80,7 +80,7 @@ For example: res = ddf.map_partitions(my_function, **kwargs) -Daft implements two APIs for mapping computations over the data in your DataFrame in parallel: :doc:`Expressions <../user_guide/basic_concepts/expressions>` and :doc:`UDFs <../user_guide/daft_in_depth/udf>`. Expressions are most useful when you need to define computation over your columns. +Daft implements two APIs for mapping computations over the data in your DataFrame in parallel: :doc:`Expressions <../user_guide/expressions>` and :doc:`UDFs <../user_guide/udf>`. Expressions are most useful when you need to define computation over your columns. .. code:: python @@ -113,7 +113,7 @@ Daft is built as a DataFrame API for distributed Machine learning. You can use D Daft supports Multimodal Data Types ----------------------------------- -Dask supports the same data types as pandas. Daft is built to support many more data types, including Images, nested JSON, tensors, etc. See :doc:`the documentation <../user_guide/daft_in_depth/datatypes>` for a list of all supported data types. +Dask supports the same data types as pandas. Daft is built to support many more data types, including Images, nested JSON, tensors, etc. See :doc:`the documentation <../user_guide/datatypes>` for a list of all supported data types. Distributed Computing and Remote Clusters ----------------------------------------- diff --git a/docs/source/user_guide/daft_in_depth/aggregations.rst b/docs/source/user_guide/aggregations.rst similarity index 100% rename from docs/source/user_guide/daft_in_depth/aggregations.rst rename to docs/source/user_guide/aggregations.rst diff --git a/docs/source/user_guide/basic_concepts.rst b/docs/source/user_guide/basic_concepts.rst index 3bb3a89023..50fb8641cc 100644 --- a/docs/source/user_guide/basic_concepts.rst +++ b/docs/source/user_guide/basic_concepts.rst @@ -1,9 +1,407 @@ Basic Concepts ============== -.. toctree:: +Daft is a distributed data engine. The main abstraction in Daft is the :class:`DataFrame `, which conceptually can be thought of as a "table" of data with rows and columns. - basic_concepts/introduction - basic_concepts/dataframe_introduction - basic_concepts/expressions - basic_concepts/read-and-write +Daft also exposes a :doc:`sql` interface which interoperates closely with the DataFrame interface, allowing you to express data transformations and queries on your tables as SQL strings. + +.. image:: /_static/daft_illustration.png + :alt: Daft python dataframes make it easy to load any data such as PDF documents, images, protobufs, csv, parquet and audio files into a table dataframe structure for easy querying + :width: 500 + :align: center + +Terminology +----------- + +DataFrames +^^^^^^^^^^ + +The :class:`DataFrame ` is the core concept in Daft. Think of it as a table with rows and columns, similar to a spreadsheet or a database table. It's designed to handle large amounts of data efficiently. + +Daft DataFrames are lazy. This means that calling most methods on a DataFrame will not execute that operation immediately - instead, DataFrames expose explicit methods such as :meth:`daft.DataFrame.show` and :meth:`daft.DataFrame.write_parquet` +which will actually trigger computation of the DataFrame. + +Expressions +^^^^^^^^^^^ + +An :class:`Expression ` is a fundamental concept in Daft that allows you to define computations on DataFrame columns. They are the building blocks for transforming and manipulating data +within your DataFrame and will be your best friend if you are working with Daft primarily using the Python API. + +Query Plan +^^^^^^^^^^ + +As mentioned earlier, Daft DataFrames are lazy. Under the hood, each DataFrame in Daft is represented by a plan of operations that describes how to compute that DataFrame. + +This plan is called the "query plan" and calling methods on the DataFrame actually adds steps to the query plan! + +When your DataFrame is executed, Daft will read this plan, optimize it to make it run faster and then execute it to compute the requested results. + +Structured Query Language (SQL) +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +SQL is a common query language for expressing queries over tables of data. Daft exposes a SQL API as an alternative (but often also complementary API) to the Python :class:`DataFrame ` and +:class:`Expression ` APIs for building queries. + +You can use SQL in Daft via the :func:`daft.sql` function, and Daft will also convert many SQL-compatible strings into Expressions via :func:`daft.sql_expr` for easy interoperability with DataFrames. + +DataFrame +--------- + +If you are coming from other DataFrame libraries such as Pandas or Polars, here are some key differences about Daft DataFrames: + +1. **Distributed:** When running in a distributed cluster, Daft splits your data into smaller "chunks" called *Partitions*. This allows Daft to process your data in parallel across multiple machines, leveraging more resources to work with large datasets. + +2. **Lazy:** When you write operations on a DataFrame, Daft doesn't execute them immediately. Instead, it creates a plan (called a query plan) of what needs to be done. This plan is optimized and only executed when you specifically request the results, which can lead to more efficient computations. + +3. **Multimodal:** Unlike traditional tables that usually contain simple data types like numbers and text, Daft DataFrames can handle complex data types in its columns. This includes things like images, audio files, or even custom Python objects. + +Common data operations that you would perform on DataFrames are: + +1. **Filtering rows:** Use :meth:`df.where(...) ` to keep only the rows that meet certain conditions. +2. **Creating new columns:** Use :meth:`df.with_column(...) ` to add a new column based on calculations from existing ones. +3. **Joining tables:** Use :meth:`df.join(other_df, ...) ` to combine two DataFrames based on common columns. +4. **Sorting:** Use :meth:`df.sort(...) ` to arrange your data based on values in one or more columns. +5. **Grouping and aggregating:** Use :meth:`df.groupby(...).agg(...) ` to summarize your data by groups. + +Creating a Dataframe +^^^^^^^^^^^^^^^^^^^^ + +.. seealso:: + + :doc:`read-and-write` - a more in-depth guide on various options for reading/writing data to/from Daft DataFrames from in-memory data (Python, Arrow), files (Parquet, CSV, JSON), SQL Databases and Data Catalogs + +Let's create our first Dataframe from a Python dictionary of columns. + +.. tabs:: + + .. group-tab:: 🐍 Python + + .. code:: python + + import daft + + df = daft.from_pydict({ + "A": [1, 2, 3, 4], + "B": [1.5, 2.5, 3.5, 4.5], + "C": [True, True, False, False], + "D": [None, None, None, None], + }) + +Examine your Dataframe by printing it: + +.. code:: python + + df + +.. code-block:: text + :caption: Output + + ╭───────┬─────────┬─────────┬──────╮ + │ A ┆ B ┆ C ┆ D │ + │ --- ┆ --- ┆ --- ┆ --- │ + │ Int64 ┆ Float64 ┆ Boolean ┆ Null │ + ╞═══════╪═════════╪═════════╪══════╡ + │ 1 ┆ 1.5 ┆ true ┆ None │ + ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌┤ + │ 2 ┆ 2.5 ┆ true ┆ None │ + ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌┤ + │ 3 ┆ 3.5 ┆ false ┆ None │ + ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌┤ + │ 4 ┆ 4.5 ┆ false ┆ None │ + ╰───────┴─────────┴─────────┴──────╯ + + (Showing first 4 of 4 rows) + + +Congratulations - you just created your first DataFrame! It has 4 columns, "A", "B", "C", and "D". Let's try to select only the "A", "B", and "C" columns: + +.. tabs:: + + .. group-tab:: 🐍 Python + + .. code:: python + + df = df.select("A", "B", "C") + df + + .. group-tab:: ⚙️ SQL + + .. code:: python + + df = daft.sql("SELECT A, B, C FROM df") + df + +.. code-block:: text + :caption: Output + + ╭───────┬─────────┬─────────╮ + │ A ┆ B ┆ C │ + │ --- ┆ --- ┆ --- │ + │ Int64 ┆ Float64 ┆ Boolean │ + ╰───────┴─────────┴─────────╯ + + (No data to display: Dataframe not materialized) + + +But wait - why is it printing the message ``(No data to display: Dataframe not materialized)`` and where are the rows of each column? + +Executing our DataFrame and Viewing Data +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +The reason that our DataFrame currently does not display its rows is that Daft DataFrames are **lazy**. This just means that Daft DataFrames will defer all its work until you tell it to execute. + +In this case, Daft is just deferring the work required to read the data and select columns, however in practice this laziness can be very useful for helping Daft optimize your queries before execution! + +.. NOTE:: + + When you call methods on a Daft Dataframe, it defers the work by adding to an internal "plan". You can examine the current plan of a DataFrame by calling :meth:`df.explain() `! + + Passing the ``show_all=True`` argument will show you the plan after Daft applies its query optimizations and the physical (lower-level) plan. + + .. code-block:: text + :caption: Plan Output + + == Unoptimized Logical Plan == + + * Project: col(A), col(B), col(C) + | + * Source: + | Number of partitions = 1 + | Output schema = A#Int64, B#Float64, C#Boolean, D#Null + + + == Optimized Logical Plan == + + * Project: col(A), col(B), col(C) + | + * Source: + | Number of partitions = 1 + | Output schema = A#Int64, B#Float64, C#Boolean, D#Null + + + == Physical Plan == + + * Project: col(A), col(B), col(C) + | Clustering spec = { Num partitions = 1 } + | + * InMemoryScan: + | Schema = A#Int64, B#Float64, C#Boolean, D#Null, + | Size bytes = 65, + | Clustering spec = { Num partitions = 1 } + +We can tell Daft to execute our DataFrame and store the results in-memory using :meth:`df.collect() `: + +.. tabs:: + + .. group-tab:: 🐍 Python + + .. code:: python + + df.collect() + df + +.. code-block:: text + :caption: Output + + ╭───────┬─────────┬─────────┬──────╮ + │ A ┆ B ┆ C ┆ D │ + │ --- ┆ --- ┆ --- ┆ --- │ + │ Int64 ┆ Float64 ┆ Boolean ┆ Null │ + ╞═══════╪═════════╪═════════╪══════╡ + │ 1 ┆ 1.5 ┆ true ┆ None │ + ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌┤ + │ 2 ┆ 2.5 ┆ true ┆ None │ + ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌┤ + │ 3 ┆ 3.5 ┆ false ┆ None │ + ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌┤ + │ 4 ┆ 4.5 ┆ false ┆ None │ + ╰───────┴─────────┴─────────┴──────╯ + + (Showing first 4 of 4 rows) + +Now your DataFrame object ``df`` is **materialized** - Daft has executed all the steps required to compute the results, and has cached the results in memory so that it can display this preview. + +Any subsequent operations on ``df`` will avoid recomputations, and just use this materialized result! + +When should I materialize my DataFrame? +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +If you "eagerly" call :meth:`df.collect() ` immediately on every DataFrame, you may run into issues: + +1. If data is too large at any step, materializing all of it may cause memory issues +2. Optimizations are not possible since we cannot "predict future operations" + +However, data science is all about experimentation and trying different things on the same data. This means that materialization is crucial when working interactively with DataFrames, since it speeds up all subsequent experimentation on that DataFrame. + +We suggest materializing DataFrames using :meth:`df.collect() ` when they contain expensive operations (e.g. sorts or expensive function calls) and have to be called multiple times by downstream code: + +.. tabs:: + + .. group-tab:: 🐍 Python + + .. code:: python + + df = df.sort("A") # expensive sort + df.collect() # materialize the DataFrame + + # All subsequent work on df avoids recomputing previous steps + df.sum("B").show() + df.mean("B").show() + df.with_column("try_this", df["A"] + 1).show(5) + + .. group-tab:: ⚙️ SQL + + .. code:: python + + df = daft.sql("SELECT * FROM df ORDER BY A") + df.collect() + + # All subsequent work on df avoids recomputing previous steps + daft.sql("SELECT sum(B) FROM df").show() + daft.sql("SELECT mean(B) FROM df").show() + daft.sql("SELECT *, (A + 1) AS try_this FROM df").show(5) + +.. code-block:: text + :caption: Output + + ╭─────────╮ + │ B │ + │ --- │ + │ Float64 │ + ╞═════════╡ + │ 12 │ + ╰─────────╯ + + (Showing first 1 of 1 rows) + + ╭─────────╮ + │ B │ + │ --- │ + │ Float64 │ + ╞═════════╡ + │ 3 │ + ╰─────────╯ + + (Showing first 1 of 1 rows) + + ╭───────┬─────────┬─────────┬──────────╮ + │ A ┆ B ┆ C ┆ try_this │ + │ --- ┆ --- ┆ --- ┆ --- │ + │ Int64 ┆ Float64 ┆ Boolean ┆ Int64 │ + ╞═══════╪═════════╪═════════╪══════════╡ + │ 1 ┆ 1.5 ┆ true ┆ 2 │ + ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌╌┤ + │ 2 ┆ 2.5 ┆ true ┆ 3 │ + ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌╌┤ + │ 3 ┆ 3.5 ┆ false ┆ 4 │ + ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌╌┤ + │ 4 ┆ 4.5 ┆ false ┆ 5 │ + ╰───────┴─────────┴─────────┴──────────╯ + + (Showing first 4 of 4 rows) + + +In many other cases however, there are better options than materializing your entire DataFrame with :meth:`df.collect() `: + +1. **Peeking with df.show(N)**: If you only want to "peek" at the first few rows of your data for visualization purposes, you can use :meth:`df.show(N) `, which processes and shows only the first ``N`` rows. +2. **Writing to disk**: The ``df.write_*`` methods will process and write your data to disk per-partition, avoiding materializing it all in memory at once. +3. **Pruning data**: You can materialize your DataFrame after performing a :meth:`df.limit() `, :meth:`df.where() ` or :meth:`df.select() ` operation which processes your data or prune it down to a smaller size. + +Schemas and Types +^^^^^^^^^^^^^^^^^ + +Notice also that when we printed our DataFrame, Daft displayed its **schema**. Each column of your DataFrame has a **name** and a **type**, and all data in that column will adhere to that type! + +Daft can display your DataFrame's schema without materializing it. Under the hood, it performs intelligent sampling of your data to determine the appropriate schema, and if you make any modifications to your DataFrame it can infer the resulting types based on the operation. + +.. NOTE:: + + Under the hood, Daft represents data in the `Apache Arrow `_ format, which allows it to efficiently represent and work on data using high-performance kernels which are written in Rust. + + +Running Computation with Expressions +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +To run computations on data in our DataFrame, we use Expressions. + +The following statement will :meth:`df.show() ` a DataFrame that has only one column - the column ``A`` from our original DataFrame but with every row incremented by 1. + +.. tabs:: + + .. group-tab:: 🐍 Python + + .. code:: python + + df.select(df["A"] + 1).show() + + .. group-tab:: ⚙️ SQL + + .. code:: python + + daft.sql("SELECT A + 1 FROM df").show() + +.. code-block:: text + :caption: Output + + ╭───────╮ + │ A │ + │ --- │ + │ Int64 │ + ╞═══════╡ + │ 2 │ + ├╌╌╌╌╌╌╌┤ + │ 3 │ + ├╌╌╌╌╌╌╌┤ + │ 4 │ + ├╌╌╌╌╌╌╌┤ + │ 5 │ + ╰───────╯ + + (Showing first 4 of 4 rows) + +.. NOTE:: + + A common pattern is to create a new columns using ``DataFrame.with_column``: + + .. tabs:: + + .. group-tab:: 🐍 Python + + .. code:: python + + # Creates a new column named "foo" which takes on values + # of column "A" incremented by 1 + df = df.with_column("foo", df["A"] + 1) + df.show() + + .. group-tab:: ⚙️ SQL + + .. code:: python + + # Creates a new column named "foo" which takes on values + # of column "A" incremented by 1 + df = daft.sql("SELECT *, A + 1 AS foo FROM df") + df.show() + +.. code-block:: text + :caption: Output + + ╭───────┬─────────┬─────────┬───────╮ + │ A ┆ B ┆ C ┆ foo │ + │ --- ┆ --- ┆ --- ┆ --- │ + │ Int64 ┆ Float64 ┆ Boolean ┆ Int64 │ + ╞═══════╪═════════╪═════════╪═══════╡ + │ 1 ┆ 1.5 ┆ true ┆ 2 │ + ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌┤ + │ 2 ┆ 2.5 ┆ true ┆ 3 │ + ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌┤ + │ 3 ┆ 3.5 ┆ false ┆ 4 │ + ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌┤ + │ 4 ┆ 4.5 ┆ false ┆ 5 │ + ╰───────┴─────────┴─────────┴───────╯ + + (Showing first 4 of 4 rows) + +Congratulations, you have just written your first **Expression**: ``df["A"] + 1``! + +Expressions are a powerful way of describing computation on columns. For more details, check out the next section on :doc:`expressions` diff --git a/docs/source/user_guide/basic_concepts/dataframe_introduction.rst b/docs/source/user_guide/basic_concepts/dataframe_introduction.rst deleted file mode 100644 index 7e1075b34b..0000000000 --- a/docs/source/user_guide/basic_concepts/dataframe_introduction.rst +++ /dev/null @@ -1,203 +0,0 @@ -Dataframe -========= - -Data in Daft is represented as a DataFrame, which is a collection of data organized as a **table** with **rows** and **columns**. - -.. image:: /_static/daft_illustration.png - :alt: Daft python dataframes make it easy to load any data such as PDF documents, images, protobufs, csv, parquet and audio files into a table dataframe structure for easy querying - :width: 500 - :align: center - -This document provides an introduction to the Daft Dataframe. - -Creating a Dataframe --------------------- - -Let's create our first Dataframe from a Python dictionary of columns. - -.. code:: python - - import daft - - df = daft.from_pydict({ - "A": [1, 2, 3, 4], - "B": [1.5, 2.5, 3.5, 4.5], - "C": [True, True, False, False], - "D": [None, None, None, None], - }) - -Examine your Dataframe by printing it: - -.. code:: python - - df - -.. code:: none - - +---------+-----------+-----------+-----------+ - | A | B | C | D | - | Int64 | Float64 | Boolean | Null | - +=========+===========+===========+===========+ - | 1 | 1.5 | true | None | - +---------+-----------+-----------+-----------+ - | 2 | 2.5 | true | None | - +---------+-----------+-----------+-----------+ - | 3 | 3.5 | false | None | - +---------+-----------+-----------+-----------+ - | 4 | 4.5 | false | None | - +---------+-----------+-----------+-----------+ - (Showing first 4 of 4 rows) - - -Congratulations - you just created your first DataFrame! It has 4 columns, "A", "B", "C", and "D". Let's try to select only the "A", "B", and "C" columns: - -.. code:: python - - df.select("A", "B", "C") - -.. code:: none - - +---------+-----------+-----------+ - | A | B | C | - | Int64 | Float64 | Boolean | - +=========+===========+===========+ - +---------+-----------+-----------+ - (No data to display: Dataframe not materialized) - - -But wait - why is it printing the message ``(No data to display: Dataframe not materialized)`` and where are the rows of each column? - -Executing our DataFrame and Viewing Data ----------------------------------------- - -The reason that our DataFrame currently does not display its rows is that Daft DataFrames are **lazy**. This just means that Daft DataFrames will defer all its work until you tell it to execute. - -In this case, Daft is just deferring the work required to read the data and select columns, however in practice this laziness can be very useful for helping Daft optimize your queries before execution! - -.. NOTE:: - - When you call methods on a Daft Dataframe, it defers the work by adding to an internal "plan". You can examine the current plan of a DataFrame by calling :meth:`df.explain() `! - - Passing the ``show_all=True`` argument will show you the plan after Daft applies its query optimizations and the physical (lower-level) plan. - -We can tell Daft to execute our DataFrame and cache the results using :meth:`df.collect() `: - -.. code:: python - - df.collect() - df - -.. code:: none - - +---------+-----------+-----------+ - | A | B | C | - | Int64 | Float64 | Boolean | - +=========+===========+===========+ - | 1 | 1.5 | true | - +---------+-----------+-----------+ - | 2 | 2.5 | true | - +---------+-----------+-----------+ - | 3 | 3.5 | false | - +---------+-----------+-----------+ - | 4 | 4.5 | false | - +---------+-----------+-----------+ - (Showing first 4 of 4 rows) - -Now your DataFrame object ``df`` is **materialized** - Daft has executed all the steps required to compute the results, and has cached the results in memory so that it can display this preview. - -Any subsequent operations on ``df`` will avoid recomputations, and just use this materialized result! - -When should I materialize my DataFrame? -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ - -If you "eagerly" call :meth:`df.collect() ` immediately on every DataFrame, you may run into issues: - -1. If data is too large at any step, materializing all of it may cause memory issues -2. Optimizations are not possible since we cannot "predict future operations" - -However, data science is all about experimentation and trying different things on the same data. This means that materialization is crucial when working interactively with DataFrames, since it speeds up all subsequent experimentation on that DataFrame. - -We suggest materializing DataFrames using :meth:`df.collect() ` when they contain expensive operations (e.g. sorts or expensive function calls) and have to be called multiple times by downstream code: - -.. code:: python - - df = df.with_column("A", df["A"].apply(expensive_function)) # expensive function - df = df.sort("A") # expensive sort - df.collect() # materialize the DataFrame - - # All subsequent work on df avoids recomputing previous steps - df.sum().show() - df.mean().show() - df.with_column("try_this", df["A"] + 1).show(5) - -In many other cases however, there are better options than materializing your entire DataFrame with :meth:`df.collect() `: - -1. **Peeking with df.show(N)**: If you only want to "peek" at the first few rows of your data for visualization purposes, you can use :meth:`df.show(N) `, which processes and shows only the first ``N`` rows. -2. **Writing to disk**: The ``df.write_*`` methods will process and write your data to disk per-partition, avoiding materializing it all in memory at once. -3. **Pruning data**: You can materialize your DataFrame after performing a :meth:`df.limit() `, :meth:`df.where() ` or :meth:`df.select() ` operation which processes your data or prune it down to a smaller size. - -Schemas and Types ------------------ - -Notice also that when we printed our DataFrame, Daft displayed its **schema**. Each column of your DataFrame has a **name** and a **type**, and all data in that column will adhere to that type! - -Daft can display your DataFrame's schema without materializing it. Under the hood, it performs intelligent sampling of your data to determine the appropriate schema, and if you make any modifications to your DataFrame it can infer the resulting types based on the operation. - -.. NOTE:: - - Under the hood, Daft represents data in the `Apache Arrow `_ format, which allows it to efficiently represent and work on data using high-performance kernels which are written in Rust. - - -Running Computations --------------------- - -To run computations on data in our DataFrame, we use Expressions. - -The following statement will :meth:`df.show() ` a DataFrame that has only one column - the column ``A`` from our original DataFrame but with every row incremented by 1. - -.. code:: python - - df.select(df["A"] + 1).show() - -.. code:: none - - +---------+ - | A | - | Int64 | - +=========+ - | 2 | - +---------+ - | 3 | - +---------+ - | 4 | - +---------+ - | 5 | - +---------+ - (Showing first 4 rows) - -.. NOTE:: - - A common pattern is to create a new columns using ``DataFrame.with_column``: - - .. code:: python - - # Creates a new column named "foo" which takes on values - # of column "A" incremented by 1 - df = df.with_column("foo", df["A"] + 1) - -Congratulations, you have just written your first **Expression**: ``df["A"] + 1``! - -Expressions -^^^^^^^^^^^ - -Expressions are how you define computations on your columns in Daft. - -The world of Daft contains much more than just numbers, and you can do much more than just add numbers together. Daft's rich Expressions API allows you to do things such as: - -1. Convert between different types with :meth:`df["numbers"].cast(float) ` -2. Download Bytes from a column containing String URLs using :meth:`df["urls"].url.download() ` -3. Run arbitrary Python functions on your data using :meth:`df["objects"].apply(my_python_function) ` - -We are also constantly looking to improve Daft and add more Expression functionality. Please contribute to the project with your ideas and code if you have an Expression in mind! - -The next section on :doc:`expressions` will provide a much deeper look at the Expressions that Daft provides. diff --git a/docs/source/user_guide/basic_concepts/expressions.rst b/docs/source/user_guide/basic_concepts/expressions.rst deleted file mode 100644 index db62ddb2fb..0000000000 --- a/docs/source/user_guide/basic_concepts/expressions.rst +++ /dev/null @@ -1,343 +0,0 @@ -Expressions -=========== - -Expressions are how you can express computations that should be run over columns of data. - -Creating Expressions --------------------- - -Referring to a column in a DataFrame -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ - -Most commonly you will be creating expressions by referring to a column from an existing DataFrame. - -To do so, simply index a DataFrame with the string name of the column: - -.. code:: python - - import daft - - df = daft.from_pydict({"A": [1, 2, 3]}) - - # Refers to column "A" in `df` - df["A"] - -.. code:: none - - col(A) - -When we evaluate this ``df["A"]`` Expression, it will evaluate to the column from the ``df`` DataFrame with name "A"! - -Refer to a column with a certain name -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ - -You may also find it necessary in certain situations to create an Expression with just the name of a column, without having an existing DataFrame to refer to. You can do this with the :func:`~daft.expressions.col` helper: - -.. code:: python - - from daft import col - - # Refers to a column named "A" - col("A") - -When this Expression is evaluated, it will resolve to "the column named A" in whatever evaluation context it is used within! - -Refer to multiple columns using a wildcard -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ - -You can create expressions on multiple columns at once using a wildcard. The expression `col("*")` selects every column in a DataFrame, and you can operate on this expression in the same way as a single column: - -.. code:: python - - import daft - from daft import col - - df = daft.from_pydict({"A": [1, 2, 3], "B": [4, 5, 6]}) - df.select(col("*") * 3).show() - -.. code:: none - - ╭───────┬───────╮ - │ A ┆ B │ - │ --- ┆ --- │ - │ Int64 ┆ Int64 │ - ╞═══════╪═══════╡ - │ 3 ┆ 12 │ - ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌┤ - │ 6 ┆ 15 │ - ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌┤ - │ 9 ┆ 18 │ - ╰───────┴───────╯ - -Literals -^^^^^^^^ - -You may find yourself needing to hardcode a "single value" oftentimes as an expression. Daft provides a :func:`~daft.expressions.lit` helper to do so: - -.. code:: python - - from daft import lit - - # Refers to an expression which always evaluates to 42 - lit(42) - -This special :func:`~daft.expressions.lit` expression we just created evaluates always to the value ``42``. - -.. _userguide-numeric-expressions: - -Numeric Expressions -------------------- - -Since column "A" is an integer, we can run numeric computation such as addition, division and checking its value. Here are some examples where we create new columns using the results of such computations: - -.. code:: python - - # Add 1 to each element in column "A" - df = df.with_column("A_add_one", df["A"] + 1) - - # Divide each element in column A by 2 - df = df.with_column("A_divide_two", df["A"] / 2.) - - # Check if each element in column A is more than 1 - df = df.with_column("A_gt_1", df["A"] > 1) - - df.collect() - -.. code:: none - - +---------+-------------+----------------+-----------+ - | A | A_add_one | A_divide_two | A_gt_1 | - | Int64 | Int64 | Float64 | Boolean | - +=========+=============+================+===========+ - | 1 | 2 | 0.5 | false | - +---------+-------------+----------------+-----------+ - | 2 | 3 | 1 | true | - +---------+-------------+----------------+-----------+ - | 3 | 4 | 1.5 | true | - +---------+-------------+----------------+-----------+ - (Showing first 3 of 3 rows) - -Notice that the returned types of these operations are also well-typed according to their input types. For example, calling ``df["A"] > 1`` returns a column of type :meth:`Boolean `. - -Both the :meth:`Float ` and :meth:`Int ` types are numeric types, and inherit many of the same arithmetic Expression operations. You may find the full list of numeric operations in the :ref:`Expressions API reference `. - -.. _userguide-string-expressions: - -String Expressions ------------------- - -Daft also lets you have columns of strings in a DataFrame. Let's take a look! - -.. code:: python - - df = daft.from_pydict({"B": ["foo", "bar", "baz"]}) - df.show() - -.. code:: none - - +--------+ - | B | - | Utf8 | - +========+ - | foo | - +--------+ - | bar | - +--------+ - | baz | - +--------+ - (Showing first 3 rows) - -Unlike the numeric types, the string type does not support arithmetic operations such as ``*`` and ``/``. The one exception to this is the ``+`` operator, which is overridden to concatenate two string expressions as is commonly done in Python. Let's try that! - -.. code:: python - - df = df.with_column("B2", df["B"] + "foo") - df.show() - -.. code:: none - - +--------+--------+ - | B | B2 | - | Utf8 | Utf8 | - +========+========+ - | foo | foofoo | - +--------+--------+ - | bar | barfoo | - +--------+--------+ - | baz | bazfoo | - +--------+--------+ - (Showing first 3 rows) - -There are also many string operators that are accessed through a separate :meth:`.str.* ` "method namespace". - -For example, to check if each element in column "B" contains the substring "a", we can use the :meth:`.str.contains ` method: - -.. code:: python - - df = df.with_column("B2_contains_B", df["B2"].str.contains(df["B"])) - df.show() - -.. code:: none - - +--------+--------+-----------------+ - | B | B2 | B2_contains_B | - | Utf8 | Utf8 | Boolean | - +========+========+=================+ - | foo | foofoo | true | - +--------+--------+-----------------+ - | bar | barfoo | true | - +--------+--------+-----------------+ - | baz | bazfoo | true | - +--------+--------+-----------------+ - (Showing first 3 rows) - -You may find a full list of string operations in the :ref:`Expressions API reference `. - -URL Expressions -^^^^^^^^^^^^^^^ - -One special case of a String column you may find yourself working with is a column of URL strings. - -Daft provides the :meth:`.url.* ` method namespace with functionality for working with URL strings. For example, to download data from URLs: - -.. code:: python - - df = daft.from_pydict({ - "urls": [ - "https://www.google.com", - "s3://daft-public-data/open-images/validation-images/0001eeaf4aed83f9.jpg", - ], - }) - df = df.with_column("data", df["urls"].url.download()) - df.collect() - -.. code:: none - - +----------------------+----------------------+ - | urls | data | - | Utf8 | Binary | - +======================+======================+ - | https://www.google.c | b'`_ as the underlying executor, so you can find the full list of supported filters in the `jaq documentation `_. - -.. _userguide-logical-expressions: - -Logical Expressions -------------------- - -Logical Expressions are an expression that refers to a column of type :meth:`Boolean `, and can only take on the values True or False. - -.. code:: python - - df = daft.from_pydict({"C": [True, False, True]}) - df["C"] - -Daft supports logical operations such as ``&`` (and) and ``|`` (or) between logical expressions. - -Comparisons -^^^^^^^^^^^ - -Many of the types in Daft support comparisons between expressions that returns a Logical Expression. - -For example, here we can compare if each element in column "A" is equal to elements in column "B": - -.. code:: python - - df = daft.from_pydict({"A": [1, 2, 3], "B": [1, 2, 4]}) - - df = df.with_column("A_eq_B", df["A"] == df["B"]) - - df.collect() - -.. code:: none - - +---------+---------+-----------+ - | A | B | A_eq_B | - | Int64 | Int64 | Boolean | - +=========+=========+===========+ - | 1 | 1 | true | - +---------+---------+-----------+ - | 2 | 2 | true | - +---------+---------+-----------+ - | 3 | 4 | false | - +---------+---------+-----------+ - (Showing first 3 of 3 rows) - -Other useful comparisons can be found in the :ref:`Expressions API reference `. - -If Else Pattern -^^^^^^^^^^^^^^^ - -The :meth:`.if_else() ` method is a useful expression to have up your sleeve for choosing values between two other expressions based on a logical expression: - -.. code:: python - - df = daft.from_pydict({"A": [1, 2, 3], "B": [0, 2, 4]}) - - # Pick values from column A if the value in column A is bigger - # than the value in column B. Otherwise, pick values from column B. - df = df.with_column( - "A_if_bigger_else_B", - (df["A"] > df["B"]).if_else(df["A"], df["B"]), - ) - - df.collect() - -.. code:: none - - +---------+---------+----------------------+ - | A | B | A_if_bigger_else_B | - | Int64 | Int64 | Int64 | - +=========+=========+======================+ - | 1 | 0 | 1 | - +---------+---------+----------------------+ - | 2 | 2 | 2 | - +---------+---------+----------------------+ - | 3 | 4 | 4 | - +---------+---------+----------------------+ - (Showing first 3 of 3 rows) - -This is a useful expression for cleaning your data! diff --git a/docs/source/user_guide/basic_concepts/introduction.rst b/docs/source/user_guide/basic_concepts/introduction.rst deleted file mode 100644 index 2fa1c8fa94..0000000000 --- a/docs/source/user_guide/basic_concepts/introduction.rst +++ /dev/null @@ -1,92 +0,0 @@ -Introduction -============ - -Daft is a distributed query engine with a DataFrame API. The two key concepts to Daft are: - -1. :class:`DataFrame `: a Table-like structure that represents rows and columns of data -2. :class:`Expression `: a symbolic representation of computation that transforms columns of the DataFrame to a new one. - -With Daft, you create :class:`DataFrame ` from a variety of sources (e.g. reading data from files, data catalogs or from Python dictionaries) and use :class:`Expression ` to manipulate data in that DataFrame. Let's take a closer look at these two abstractions! - -DataFrame ---------- - -Conceptually, a DataFrame is a "table" of data, with rows and columns. - -.. image:: /_static/daft_illustration.png - :alt: Daft python dataframes make it easy to load any data such as PDF documents, images, protobufs, csv, parquet and audio files into a table dataframe structure for easy querying - :width: 500 - :align: center - -Using this abstraction of a DataFrame, you can run common tabular operations such as: - -1. Filtering rows: :meth:`df.where(...) ` -2. Creating new columns as a computation of existing columns: :meth:`df.with_column(...) ` -3. Joining two tables together: :meth:`df.join(...) ` -4. Sorting a table by the values in specified column(s): :meth:`df.sort(...) ` -5. Grouping and aggregations: :meth:`df.groupby(...).agg(...) ` - -Daft DataFrames are: - -1. **Distributed:** your data is split into *Partitions* and can be processed in parallel/on different machines -2. **Lazy:** computations are enqueued in a query plan which is then optimized and executed only when requested -3. **Multimodal:** columns can contain complex datatypes such as tensors, images and Python objects - -Since Daft is lazy, it can actually execute the query plan on a variety of different backends. By default, it will run computations locally using Python multithreading. However if you need to scale to large amounts of data that cannot be processed on a single machine, using the Ray runner allows Daft to run computations on a `Ray `_ cluster instead. - -Expressions ------------ - -The other important concept to understand when working with Daft are **expressions**. - -Because Daft is "lazy", it needs a way to represent computations that need to be performed on its data so that it can execute these computations at some later time. The answer to this is an :class:`~daft.expressions.Expression`! - -The simplest Expressions are: - -1. The column expression: :func:`col("a") ` which is used to refer to "some column named 'a'" -2. Or, if you already have an existing DataFrame ``df`` with a column named "a", you can refer to its column with Python's square bracket indexing syntax: ``df["a"]`` -3. The literal expression: :func:`lit(100) ` which represents a column that always takes on the provided value - -Daft then provides an extremely rich Expressions library to allow you to compose different computations that need to happen. For example: - -.. code:: python - - from daft import col, DataType - - # Take the column named "a" and add 1 to each element - col("a") + 1 - - # Take the column named "a", cast it to a string and check each element, returning True if it starts with "1" - col("a").cast(DataType.string()).str.startswith("1") - -Expressions are used in DataFrame operations, and the names of these Expressions are resolved to column names on the DataFrame that they are running on. Here is an example: - -.. code:: python - - import daft - - # Create a dataframe with a column "a" that has values [1, 2, 3] - df = daft.from_pydict({"a": [1, 2, 3]}) - - # Create new columns called "a_plus_1" and "a_startswith_1" using Expressions - df = df.select( - col("a"), - (col("a") + 1).alias("a_plus_1"), - col("a").cast(DataType.string()).str.startswith("1").alias("a_startswith_1"), - ) - - df.show() - -.. code:: none - - +---------+------------+------------------+ - | a | a_plus_1 | a_startswith_1 | - | Int64 | Int64 | Boolean | - +=========+============+==================+ - | 1 | 2 | true | - +---------+------------+------------------+ - | 2 | 3 | false | - +---------+------------+------------------+ - | 3 | 4 | false | - +---------+------------+------------------+ - (Showing first 3 rows) diff --git a/docs/source/user_guide/daft_in_depth.rst b/docs/source/user_guide/daft_in_depth.rst deleted file mode 100644 index 9b9702daca..0000000000 --- a/docs/source/user_guide/daft_in_depth.rst +++ /dev/null @@ -1,9 +0,0 @@ -Daft in Depth -============= - -.. toctree:: - - daft_in_depth/datatypes - daft_in_depth/dataframe-operations - daft_in_depth/aggregations - daft_in_depth/udf diff --git a/docs/source/user_guide/daft_in_depth/dataframe-operations.rst b/docs/source/user_guide/dataframe-operations.rst similarity index 100% rename from docs/source/user_guide/daft_in_depth/dataframe-operations.rst rename to docs/source/user_guide/dataframe-operations.rst diff --git a/docs/source/user_guide/daft_in_depth/datatypes.rst b/docs/source/user_guide/datatypes.rst similarity index 100% rename from docs/source/user_guide/daft_in_depth/datatypes.rst rename to docs/source/user_guide/datatypes.rst diff --git a/docs/source/user_guide/expressions.rst b/docs/source/user_guide/expressions.rst new file mode 100644 index 0000000000..54147a9401 --- /dev/null +++ b/docs/source/user_guide/expressions.rst @@ -0,0 +1,584 @@ +Expressions +=========== + +Expressions are how you can express computations that should be run over columns of data. + +Creating Expressions +^^^^^^^^^^^^^^^^^^^^ + +Referring to a column in a DataFrame +#################################### + +Most commonly you will be creating expressions by using the :func:`daft.col` function. + +.. tabs:: + + .. group-tab:: 🐍 Python + + .. code:: python + + # Refers to column "A" + daft.col("A") + + .. group-tab:: ⚙️ SQL + + .. code:: python + + daft.sql_expr("A") + +.. code-block:: text + :caption: Output + + col(A) + +The above code creates an Expression that refers to a column named ``"A"``. + +Using SQL +######### + +Daft can also parse valid SQL as expressions. + +.. tabs:: + + .. group-tab:: ⚙️ SQL + + .. code:: python + + daft.sql_expr("A + 1") + +.. code-block:: text + :caption: Output + + col(A) + lit(1) + +The above code will create an expression representing "the column named 'x' incremented by 1". For many APIs, sql_expr will actually be applied for you as syntactic sugar! + +Literals +######## + +You may find yourself needing to hardcode a "single value" oftentimes as an expression. Daft provides a :func:`~daft.expressions.lit` helper to do so: + +.. tabs:: + + .. group-tab:: 🐍 Python + + .. code:: python + + from daft import lit + + # Refers to an expression which always evaluates to 42 + lit(42) + + .. group-tab:: ⚙️ SQL + + .. code:: python + + # Refers to an expression which always evaluates to 42 + daft.sql_expr("42") + +.. code-block:: text + :caption: Output + + lit(42) + +This special :func:`~daft.expressions.lit` expression we just created evaluates always to the value ``42``. + +Wildcard Expressions +#################### + +You can create expressions on multiple columns at once using a wildcard. The expression `col("*")` selects every column in a DataFrame, and you can operate on this expression in the same way as a single column: + +.. tabs:: + + .. group-tab:: 🐍 Python + + .. code:: python + + import daft + from daft import col + + df = daft.from_pydict({"A": [1, 2, 3], "B": [4, 5, 6]}) + df.select(col("*") * 3).show() + +.. code-block:: text + :caption: Output + + ╭───────┬───────╮ + │ A ┆ B │ + │ --- ┆ --- │ + │ Int64 ┆ Int64 │ + ╞═══════╪═══════╡ + │ 3 ┆ 12 │ + ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌┤ + │ 6 ┆ 15 │ + ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌┤ + │ 9 ┆ 18 │ + ╰───────┴───────╯ + +Wildcards also work very well for accessing all members of a struct column: + + +.. tabs:: + + .. group-tab:: 🐍 Python + + .. code:: python + + import daft + from daft import col + + df = daft.from_pydict({ + "person": [ + {"name": "Alice", "age": 30}, + {"name": "Bob", "age": 25}, + {"name": "Charlie", "age": 35} + ] + }) + + # Access all fields of the 'person' struct + df.select(col("person.*")).show() + + .. group-tab:: ⚙️ SQL + + .. code:: python + + import daft + + df = daft.from_pydict({ + "person": [ + {"name": "Alice", "age": 30}, + {"name": "Bob", "age": 25}, + {"name": "Charlie", "age": 35} + ] + }) + + # Access all fields of the 'person' struct using SQL + daft.sql("SELECT person.* FROM df").show() + +.. code-block:: text + :caption: Output + + ╭──────────┬───────╮ + │ name ┆ age │ + │ --- ┆ --- │ + │ String ┆ Int64 │ + ╞══════════╪═══════╡ + │ Alice ┆ 30 │ + ├╌╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌┤ + │ Bob ┆ 25 │ + ├╌╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌┤ + │ Charlie ┆ 35 │ + ╰──────────┴───────╯ + +In this example, we use the wildcard `*` to access all fields of the `person` struct column. This is equivalent to selecting each field individually (`person.name`, `person.age`), but is more concise and flexible, especially when dealing with structs that have many fields. + + + +Composing Expressions +^^^^^^^^^^^^^^^^^^^^^ + +.. _userguide-numeric-expressions: + +Numeric Expressions +################### + +Since column "A" is an integer, we can run numeric computation such as addition, division and checking its value. Here are some examples where we create new columns using the results of such computations: + +.. tabs:: + + .. group-tab:: 🐍 Python + + .. code:: python + + # Add 1 to each element in column "A" + df = df.with_column("A_add_one", df["A"] + 1) + + # Divide each element in column A by 2 + df = df.with_column("A_divide_two", df["A"] / 2.) + + # Check if each element in column A is more than 1 + df = df.with_column("A_gt_1", df["A"] > 1) + + df.collect() + + .. group-tab:: ⚙️ SQL + + .. code:: python + + df = daft.sql(""" + SELECT + *, + A + 1 AS A_add_one, + A / 2.0 AS A_divide_two, + A > 1 AS A_gt_1 + FROM df + """) + df.collect() + +.. code-block:: text + :caption: Output + + +---------+-------------+----------------+-----------+ + | A | A_add_one | A_divide_two | A_gt_1 | + | Int64 | Int64 | Float64 | Boolean | + +=========+=============+================+===========+ + | 1 | 2 | 0.5 | false | + +---------+-------------+----------------+-----------+ + | 2 | 3 | 1 | true | + +---------+-------------+----------------+-----------+ + | 3 | 4 | 1.5 | true | + +---------+-------------+----------------+-----------+ + (Showing first 3 of 3 rows) + +Notice that the returned types of these operations are also well-typed according to their input types. For example, calling ``df["A"] > 1`` returns a column of type :meth:`Boolean `. + +Both the :meth:`Float ` and :meth:`Int ` types are numeric types, and inherit many of the same arithmetic Expression operations. You may find the full list of numeric operations in the :ref:`Expressions API reference `. + +.. _userguide-string-expressions: + +String Expressions +################## + +Daft also lets you have columns of strings in a DataFrame. Let's take a look! + +.. tabs:: + + .. group-tab:: 🐍 Python + + .. code:: python + + df = daft.from_pydict({"B": ["foo", "bar", "baz"]}) + df.show() + +.. code-block:: text + :caption: Output + + +--------+ + | B | + | Utf8 | + +========+ + | foo | + +--------+ + | bar | + +--------+ + | baz | + +--------+ + (Showing first 3 rows) + +Unlike the numeric types, the string type does not support arithmetic operations such as ``*`` and ``/``. The one exception to this is the ``+`` operator, which is overridden to concatenate two string expressions as is commonly done in Python. Let's try that! + +.. tabs:: + + .. group-tab:: 🐍 Python + + .. code:: python + + df = df.with_column("B2", df["B"] + "foo") + df.show() + + .. group-tab:: ⚙️ SQL + + .. code:: python + + df = daft.sql("SELECT *, B + 'foo' AS B2 FROM df") + df.show() + +.. code-block:: text + :caption: Output + + +--------+--------+ + | B | B2 | + | Utf8 | Utf8 | + +========+========+ + | foo | foofoo | + +--------+--------+ + | bar | barfoo | + +--------+--------+ + | baz | bazfoo | + +--------+--------+ + (Showing first 3 rows) + +There are also many string operators that are accessed through a separate :meth:`.str.* ` "method namespace". + +For example, to check if each element in column "B" contains the substring "a", we can use the :meth:`.str.contains ` method: + +.. tabs:: + + .. group-tab:: 🐍 Python + + .. code:: python + + df = df.with_column("B2_contains_B", df["B2"].str.contains(df["B"])) + df.show() + + .. group-tab:: ⚙️ SQL + + .. code:: python + + df = daft.sql("SELECT *, contains(B2, B) AS B2_contains_B FROM df") + df.show() + +.. code-block:: text + :caption: Output + + +--------+--------+-----------------+ + | B | B2 | B2_contains_B | + | Utf8 | Utf8 | Boolean | + +========+========+=================+ + | foo | foofoo | true | + +--------+--------+-----------------+ + | bar | barfoo | true | + +--------+--------+-----------------+ + | baz | bazfoo | true | + +--------+--------+-----------------+ + (Showing first 3 rows) + +You may find a full list of string operations in the :ref:`Expressions API reference `. + +URL Expressions +############### + +One special case of a String column you may find yourself working with is a column of URL strings. + +Daft provides the :meth:`.url.* ` method namespace with functionality for working with URL strings. For example, to download data from URLs: + +.. tabs:: + + .. group-tab:: 🐍 Python + + .. code:: python + + df = daft.from_pydict({ + "urls": [ + "https://www.google.com", + "s3://daft-public-data/open-images/validation-images/0001eeaf4aed83f9.jpg", + ], + }) + df = df.with_column("data", df["urls"].url.download()) + df.collect() + + .. group-tab:: ⚙️ SQL + + .. code:: python + + + df = daft.from_pydict({ + "urls": [ + "https://www.google.com", + "s3://daft-public-data/open-images/validation-images/0001eeaf4aed83f9.jpg", + ], + }) + df = daft.sql(""" + SELECT + urls, + url_download(urls) AS data + FROM df + """) + df.collect() + +.. code-block:: text + :caption: Output + + +----------------------+----------------------+ + | urls | data | + | Utf8 | Binary | + +======================+======================+ + | https://www.google.c | b'`_ as the underlying executor, so you can find the full list of supported filters in the `jaq documentation `_. + +.. _userguide-logical-expressions: + +Logical Expressions +################### + +Logical Expressions are an expression that refers to a column of type :meth:`Boolean `, and can only take on the values True or False. + +.. tabs:: + + .. group-tab:: 🐍 Python + + .. code:: python + + df = daft.from_pydict({"C": [True, False, True]}) + +Daft supports logical operations such as ``&`` (and) and ``|`` (or) between logical expressions. + +Comparisons +########### + +Many of the types in Daft support comparisons between expressions that returns a Logical Expression. + +For example, here we can compare if each element in column "A" is equal to elements in column "B": + +.. tabs:: + + .. group-tab:: 🐍 Python + + .. code:: python + + df = daft.from_pydict({"A": [1, 2, 3], "B": [1, 2, 4]}) + + df = df.with_column("A_eq_B", df["A"] == df["B"]) + + df.collect() + + .. group-tab:: ⚙️ SQL + + .. code:: python + + df = daft.from_pydict({"A": [1, 2, 3], "B": [1, 2, 4]}) + + df = daft.sql(""" + SELECT + A, + B, + A = B AS A_eq_B + FROM df + """) + + df.collect() + +.. code-block:: text + :caption: Output + + ╭───────┬───────┬─────────╮ + │ A ┆ B ┆ A_eq_B │ + │ --- ┆ --- ┆ --- │ + │ Int64 ┆ Int64 ┆ Boolean │ + ╞═══════╪═══════╪═════════╡ + │ 1 ┆ 1 ┆ true │ + ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┤ + │ 2 ┆ 2 ┆ true │ + ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┤ + │ 3 ┆ 4 ┆ false │ + ╰───────┴───────┴─────────╯ + + (Showing first 3 of 3 rows) + +Other useful comparisons can be found in the :ref:`Expressions API reference `. + +If Else Pattern +############### + +The :meth:`.if_else() ` method is a useful expression to have up your sleeve for choosing values between two other expressions based on a logical expression: + +.. tabs:: + + .. group-tab:: 🐍 Python + + .. code:: python + + df = daft.from_pydict({"A": [1, 2, 3], "B": [0, 2, 4]}) + + # Pick values from column A if the value in column A is bigger + # than the value in column B. Otherwise, pick values from column B. + df = df.with_column( + "A_if_bigger_else_B", + (df["A"] > df["B"]).if_else(df["A"], df["B"]), + ) + + df.collect() + + .. group-tab:: ⚙️ SQL + + .. code:: python + + df = daft.from_pydict({"A": [1, 2, 3], "B": [0, 2, 4]}) + + df = daft.sql(""" + SELECT + A, + B, + CASE + WHEN A > B THEN A + ELSE B + END AS A_if_bigger_else_B + FROM df + """) + + df.collect() + +.. code-block:: text + :caption: Output + + ╭───────┬───────┬────────────────────╮ + │ A ┆ B ┆ A_if_bigger_else_B │ + │ --- ┆ --- ┆ --- │ + │ Int64 ┆ Int64 ┆ Int64 │ + ╞═══════╪═══════╪════════════════════╡ + │ 1 ┆ 0 ┆ 1 │ + ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌┤ + │ 2 ┆ 2 ┆ 2 │ + ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌┤ + │ 3 ┆ 4 ┆ 4 │ + ╰───────┴───────┴────────────────────╯ + + (Showing first 3 of 3 rows) + +This is a useful expression for cleaning your data! diff --git a/docs/source/user_guide/fotw/fotw-001-images.ipynb b/docs/source/user_guide/fotw/fotw-001-images.ipynb index 827f98dd57..37d1f796d2 100644 --- a/docs/source/user_guide/fotw/fotw-001-images.ipynb +++ b/docs/source/user_guide/fotw/fotw-001-images.ipynb @@ -447,7 +447,7 @@ "metadata": {}, "source": [ "### Create Thumbnails\n", - "[Expressions](../basic_concepts/expressions) are a Daft API for defining computation that needs to happen over your columns. There are dedicated `image.(...)` Expressions for working with images.\n", + "[Expressions](../expressions) are a Daft API for defining computation that needs to happen over your columns. There are dedicated `image.(...)` Expressions for working with images.\n", "\n", "You can use the `image.resize` Expression to create a thumbnail of each image:" ] @@ -527,7 +527,7 @@ "\n", "We'll define a function that uses a pre-trained PyTorch model [ResNet50](https://pytorch.org/vision/main/models/generated/torchvision.models.resnet50.html) to classify the dog pictures. We'll then pass the `image` column to this PyTorch model and send the classification predictions to a new column `classify_breed`. \n", "\n", - "You will use Daft [User-Defined Functions (UDFs)](../daft_in_depth/udf) to do this. Daft UDFs which are the best way to run computations over multiple rows or columns.\n", + "You will use Daft [User-Defined Functions (UDFs)](../udf) to do this. Daft UDFs which are the best way to run computations over multiple rows or columns.\n", "\n", "#### Setting up PyTorch\n", "\n", diff --git a/docs/source/user_guide/index.rst b/docs/source/user_guide/index.rst index e79607a84d..b4b7150215 100644 --- a/docs/source/user_guide/index.rst +++ b/docs/source/user_guide/index.rst @@ -6,7 +6,13 @@ Daft User Guide :maxdepth: 1 basic_concepts - daft_in_depth + read-and-write + expressions + datatypes + dataframe-operations + sql + aggregations + udf poweruser integrations tutorials @@ -14,22 +20,7 @@ Daft User Guide Welcome to **Daft**! -Daft is a Python dataframe library that enables Pythonic data processing at large scale. - -* **Fast** - Daft kernels are written and accelerated using Rust on Apache Arrow arrays. - -* **Flexible** - you can work with any Python object in a Daft Dataframe. - -* **Interactive** - Daft provides a first-class notebook experience. - -* **Scalable** - Daft uses out-of-core algorithms to work with datasets that cannot fit in memory. - -* **Distributed** - Daft scales to a cluster of machines using Ray to crunch terabytes of data. - -* **Intelligent** - Daft performs query optimizations to speed up your work. - -The core interface provided by Daft is the *DataFrame*, which is a table of data consisting of rows and columns. This user guide -aims to help Daft users master the usage of the Daft *DataFrame* for all your data processing needs! +This user guide aims to help Daft users master the usage of the Daft for all your data needs. .. NOTE:: @@ -39,8 +30,7 @@ aims to help Daft users master the usage of the Daft *DataFrame* for all your da code you may wish to take a look at these resources: 1. :doc:`../10-min`: Itching to run some Daft code? Hit the ground running with our 10 minute quickstart notebook. - 2. (Coming soon!) Cheatsheet: Quick reference to commonly-used Daft APIs and usage patterns - useful to keep next to your laptop as you code! - 3. :doc:`../api_docs/index`: Searchable documentation and reference material to Daft's public Python API. + 2. :doc:`../api_docs/index`: Searchable documentation and reference material to Daft's public API. Table of Contents ----------------- @@ -52,11 +42,23 @@ The Daft User Guide is laid out as follows: High-level overview of Daft interfaces and usage to give you a better understanding of how Daft will fit into your day-to-day workflow. -:doc:`Daft in Depth ` -************************************ +Daft in Depth +************* Core Daft concepts all Daft users will find useful to understand deeply. +* :doc:`read-and-write` +* :doc:`expressions` +* :doc:`datatypes` +* :doc:`dataframe-operations` +* :doc:`aggregations` +* :doc:`udf` + +:doc:`Structured Query Language (SQL) ` +******************************************** + +A look into Daft's SQL interface and how it complements Daft's Pythonic DataFrame APIs. + :doc:`The Daft Poweruser ` ************************************* diff --git a/docs/source/user_guide/basic_concepts/read-and-write.rst b/docs/source/user_guide/read-and-write.rst similarity index 64% rename from docs/source/user_guide/basic_concepts/read-and-write.rst rename to docs/source/user_guide/read-and-write.rst index 1d1a481fea..f8585111d9 100644 --- a/docs/source/user_guide/basic_concepts/read-and-write.rst +++ b/docs/source/user_guide/read-and-write.rst @@ -1,5 +1,5 @@ -Reading/Writing -=============== +Reading/Writing Data +==================== Daft can read data from a variety of sources, and write data to many destinations. @@ -37,7 +37,7 @@ To learn more about each of these constructors, as well as the options that they From Data Catalogs ^^^^^^^^^^^^^^^^^^ -If you use catalogs such as Apache Iceberg or Hive, you may wish to consult our user guide on integrations with Data Catalogs: :doc:`Daft integration with Data Catalogs <../integrations/>`. +If you use catalogs such as Apache Iceberg or Hive, you may wish to consult our user guide on integrations with Data Catalogs: :doc:`Daft integration with Data Catalogs `. From File Paths ^^^^^^^^^^^^^^^ @@ -87,7 +87,50 @@ In order to partition the data, you can specify a partition column, which will a # Read with a partition column df = daft.read_sql("SELECT * FROM my_table", partition_col="date", uri) -To learn more, consult the :doc:`SQL User Guide <../integrations/sql>` or the API documentation on :func:`daft.read_sql`. +To learn more, consult the :doc:`SQL User Guide ` or the API documentation on :func:`daft.read_sql`. + + +Reading a column of URLs +------------------------ + +Daft provides a convenient way to read data from a column of URLs using the :meth:`.url.download() ` method. This is particularly useful when you have a DataFrame with a column containing URLs pointing to external resources that you want to fetch and incorporate into your DataFrame. + +Here's an example of how to use this feature: + +.. code:: python + + # Assume we have a DataFrame with a column named 'image_urls' + df = daft.from_pydict({ + "image_urls": [ + "https://example.com/image1.jpg", + "https://example.com/image2.jpg", + "https://example.com/image3.jpg" + ] + }) + + # Download the content from the URLs and create a new column 'image_data' + df = df.with_column("image_data", df["image_urls"].url.download()) + df.show() + +.. code-block:: text + :caption: Output + + +------------------------------------+------------------------------------+ + | image_urls | image_data | + | Utf8 | Binary | + +====================================+====================================+ + | https://example.com/image1.jpg | b'\xff\xd8\xff\xe0\x00\x10JFIF...' | + +------------------------------------+------------------------------------+ + | https://example.com/image2.jpg | b'\xff\xd8\xff\xe0\x00\x10JFIF...' | + +------------------------------------+------------------------------------+ + | https://example.com/image3.jpg | b'\xff\xd8\xff\xe0\x00\x10JFIF...' | + +------------------------------------+------------------------------------+ + + (Showing first 3 of 3 rows) + + +This approach allows you to efficiently download and process data from a large number of URLs in parallel, leveraging Daft's distributed computing capabilities. + Writing Data diff --git a/docs/source/user_guide/sql.rst b/docs/source/user_guide/sql.rst new file mode 100644 index 0000000000..fec2761e05 --- /dev/null +++ b/docs/source/user_guide/sql.rst @@ -0,0 +1,244 @@ +SQL +=== + +Daft supports Structured Query Language (SQL) as a way of constructing query plans (represented in Python as a :class:`daft.DataFrame`) and expressions (:class:`daft.Expression`). + +SQL is a human-readable way of constructing these query plans, and can often be more ergonomic than using DataFrames for writing queries. + +.. NOTE:: + Daft's SQL support is new and is constantly being improved on! Please give us feedback and we'd love to hear more about what you would like. + +Running SQL on DataFrames +------------------------- + +Daft's :func:`daft.sql` function will automatically detect any :class:`daft.DataFrame` objects in your current Python environment to let you query them easily by name. + +.. tabs:: + + .. group-tab:: ⚙️ SQL + + .. code:: python + + # Note the variable name `my_special_df` + my_special_df = daft.from_pydict({"A": [1, 2, 3], "B": [1, 2, 3]}) + + # Use the SQL table name "my_special_df" to refer to the above DataFrame! + sql_df = daft.sql("SELECT A, B FROM my_special_df") + + sql_df.show() + +.. code-block:: text + :caption: Output + + ╭───────┬───────╮ + │ A ┆ B │ + │ --- ┆ --- │ + │ Int64 ┆ Int64 │ + ╞═══════╪═══════╡ + │ 1 ┆ 1 │ + ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌┤ + │ 2 ┆ 2 │ + ├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌┤ + │ 3 ┆ 3 │ + ╰───────┴───────╯ + + (Showing first 3 of 3 rows) + +In the above example, we query the DataFrame called `"my_special_df"` by simply referring to it in the SQL command. This produces a new DataFrame `sql_df` which can +natively integrate with the rest of your Daft query. + +Reading data from SQL +--------------------- + +.. WARNING:: + + This feature is a WIP and will be coming soon! We will support reading common datasources directly from SQL: + + .. code-block:: python + + daft.sql("SELECT * FROM read_parquet('s3://...')") + daft.sql("SELECT * FROM read_delta_lake('s3://...')") + + Today, a workaround for this is to construct your dataframe in Python first and use it from SQL instead: + + .. code-block:: python + + df = daft.read_parquet("s3://...") + daft.sql("SELECT * FROM df") + + We appreciate your patience with us and hope to deliver this crucial feature soon! + +SQL Expressions +--------------- + +SQL has the concept of expressions as well. Here is an example of a simple addition expression, adding columns "a" and "b" in SQL to produce a new column C. + +We also present here the equivalent query for SQL and DataFrame. Notice how similar the concepts are! + +.. tabs:: + + .. group-tab:: ⚙️ SQL + + .. code:: python + + df = daft.from_pydict({"A": [1, 2, 3], "B": [1, 2, 3]}) + df = daft.sql("SELECT A + B as C FROM df") + df.show() + + .. group-tab:: 🐍 Python + + .. code:: python + + expr = (daft.col("A") + daft.col("B")).alias("C") + + df = daft.from_pydict({"A": [1, 2, 3], "B": [1, 2, 3]}) + df = df.select(expr) + df.show() + +.. code-block:: text + :caption: Output + + ╭───────╮ + │ C │ + │ --- │ + │ Int64 │ + ╞═══════╡ + │ 2 │ + ├╌╌╌╌╌╌╌┤ + │ 4 │ + ├╌╌╌╌╌╌╌┤ + │ 6 │ + ╰───────╯ + + (Showing first 3 of 3 rows) + +In the above query, both the SQL version of the query and the DataFrame version of the query produce the same result. + +Under the hood, they run the same Expression ``col("A") + col("B")``! + +One really cool trick you can do is to use the :func:`daft.sql_expr` function as a helper to easily create Expressions. The following are equivalent: + +.. tabs:: + + .. group-tab:: ⚙️ SQL + + .. code:: python + + sql_expr = daft.sql_expr("A + B as C") + print("SQL expression:", sql_expr) + + .. group-tab:: 🐍 Python + + .. code:: python + + py_expr = (daft.col("A") + daft.col("B")).alias("C") + print("Python expression:", py_expr) + + +.. code-block:: text + :caption: Output + + SQL expression: col(A) + col(B) as C + Python expression: col(A) + col(B) as C + +This means that you can pretty much use SQL anywhere you use Python expressions, making Daft extremely versatile at mixing workflows which leverage both SQL and Python. + +As an example, consider the filter query below and compare the two equivalent Python and SQL queries: + +.. tabs:: + + .. group-tab:: ⚙️ SQL + + .. code:: python + + df = daft.from_pydict({"A": [1, 2, 3], "B": [1, 2, 3]}) + + # Daft automatically converts this string using `daft.sql_expr` + df = df.where("A < 2") + + df.show() + + .. group-tab:: 🐍 Python + + .. code:: python + + df = daft.from_pydict({"A": [1, 2, 3], "B": [1, 2, 3]}) + + # Using Daft's Python Expression API + df = df.where(df["A"] < 2) + + df.show() + +.. code-block:: text + :caption: Output + + ╭───────┬───────╮ + │ A ┆ B │ + │ --- ┆ --- │ + │ Int64 ┆ Int64 │ + ╞═══════╪═══════╡ + │ 1 ┆ 1 │ + ╰───────┴───────╯ + + (Showing first 1 of 1 rows) + +Pretty sweet! Of course, this support for running Expressions on your columns extends well beyond arithmetic as we'll see in the next section on SQL Functions. + +SQL Functions +------------- + +SQL also has access to all of Daft's powerful :class:`daft.Expression` functionality through SQL functions. + +However, unlike the Python Expression API which encourages method-chaining (e.g. ``col("a").url.download().image.decode()``), in SQL you have to do function nesting instead (e.g. ``"image_decode(url_download(a))""``). + +.. NOTE:: + + A full catalog of the available SQL Functions in Daft is available in the :doc:`../api_docs/sql`. + + Note that it closely mirrors the Python API, with some function naming differences vs the available Python methods. + We also have some aliased functions for ANSI SQL-compliance or familiarity to users coming from other common SQL dialects such as PostgreSQL and SparkSQL to easily find their functionality. + +Here is an example of an equivalent function call in SQL vs Python: + +.. tabs:: + + .. group-tab:: ⚙️ SQL + + .. code:: python + + df = daft.from_pydict({"urls": [ + "https://user-images.githubusercontent.com/17691182/190476440-28f29e87-8e3b-41c4-9c28-e112e595f558.png", + "https://user-images.githubusercontent.com/17691182/190476440-28f29e87-8e3b-41c4-9c28-e112e595f558.png", + "https://user-images.githubusercontent.com/17691182/190476440-28f29e87-8e3b-41c4-9c28-e112e595f558.png", + ]}) + df = daft.sql("SELECT image_decode(url_download(urls)) FROM df") + df.show() + + .. group-tab:: 🐍 Python + + .. code:: python + + df = daft.from_pydict({"urls": [ + "https://user-images.githubusercontent.com/17691182/190476440-28f29e87-8e3b-41c4-9c28-e112e595f558.png", + "https://user-images.githubusercontent.com/17691182/190476440-28f29e87-8e3b-41c4-9c28-e112e595f558.png", + "https://user-images.githubusercontent.com/17691182/190476440-28f29e87-8e3b-41c4-9c28-e112e595f558.png", + ]}) + df = df.select(daft.col("urls").url.download().image.decode()) + df.show() + +.. code-block:: text + :caption: Output + + ╭──────────────╮ + │ urls │ + │ --- │ + │ Image[MIXED] │ + ╞══════════════╡ + │ │ + ├╌╌╌╌╌╌╌╌╌╌╌╌╌╌┤ + │ │ + ├╌╌╌╌╌╌╌╌╌╌╌╌╌╌┤ + │ │ + ╰──────────────╯ + + (Showing first 3 of 3 rows) diff --git a/docs/source/user_guide/daft_in_depth/udf.rst b/docs/source/user_guide/udf.rst similarity index 100% rename from docs/source/user_guide/daft_in_depth/udf.rst rename to docs/source/user_guide/udf.rst diff --git a/requirements-dev.txt b/requirements-dev.txt index a67574df90..9c7809ac80 100644 --- a/requirements-dev.txt +++ b/requirements-dev.txt @@ -88,3 +88,4 @@ sphinx-book-theme==1.1.0; python_version >= "3.9" sphinx-reredirects>=0.1.1 sphinx-copybutton>=0.5.2 sphinx-autosummary-accessors==2023.4.0; python_version >= "3.9" +sphinx-tabs==3.4.5 diff --git a/src/daft-sql/src/functions.rs b/src/daft-sql/src/functions.rs index 10663cf47d..6b456af17c 100644 --- a/src/daft-sql/src/functions.rs +++ b/src/daft-sql/src/functions.rs @@ -84,6 +84,16 @@ pub trait SQLFunction: Send + Sync { } fn to_expr(&self, inputs: &[FunctionArg], planner: &SQLPlanner) -> SQLPlannerResult; + + /// Produce the docstrings for this SQL function, parametrized by an alias which is the function name to invoke this in SQL + fn docstrings(&self, alias: &str) -> String { + format!("{alias}: No docstring available") + } + + /// Produce the docstrings for this SQL function, parametrized by an alias which is the function name to invoke this in SQL + fn arg_names(&self) -> &'static [&'static str] { + &["todo"] + } } /// TODOs @@ -91,6 +101,7 @@ pub trait SQLFunction: Send + Sync { /// - Add more functions.. pub struct SQLFunctions { pub(crate) map: HashMap>, + pub(crate) docsmap: HashMap, } pub(crate) struct SQLFunctionArguments { @@ -160,6 +171,7 @@ impl SQLFunctions { pub fn new() -> Self { Self { map: HashMap::new(), + docsmap: HashMap::new(), } } @@ -170,6 +182,8 @@ impl SQLFunctions { /// Add a [FunctionExpr] to the [SQLFunctions] instance. pub fn add_fn(&mut self, name: &str, func: F) { + self.docsmap + .insert(name.to_string(), (func.docstrings(name), func.arg_names())); self.map.insert(name.to_string(), Arc::new(func)); } diff --git a/src/daft-sql/src/modules/aggs.rs b/src/daft-sql/src/modules/aggs.rs index 695d3c9c79..0fbd2f7067 100644 --- a/src/daft-sql/src/modules/aggs.rs +++ b/src/daft-sql/src/modules/aggs.rs @@ -41,6 +41,26 @@ impl SQLFunction for AggExpr { to_expr(self, inputs.as_slice()) } } + + fn docstrings(&self, alias: &str) -> String { + match self { + Self::Count(_, _) => static_docs::COUNT_DOCSTRING.to_string(), + Self::Sum(_) => static_docs::SUM_DOCSTRING.to_string(), + Self::Mean(_) => static_docs::AVG_DOCSTRING.replace("{}", alias), + Self::Min(_) => static_docs::MIN_DOCSTRING.to_string(), + Self::Max(_) => static_docs::MAX_DOCSTRING.to_string(), + e => unimplemented!("Need to implement docstrings for {e}"), + } + } + + fn arg_names(&self) -> &'static [&'static str] { + match self { + Self::Count(_, _) | Self::Sum(_) | Self::Mean(_) | Self::Min(_) | Self::Max(_) => { + &["input"] + } + e => unimplemented!("Need to implement arg names for {e}"), + } + } } fn handle_count(inputs: &[FunctionArg], planner: &SQLPlanner) -> SQLPlannerResult { @@ -103,3 +123,201 @@ pub(crate) fn to_expr(expr: &AggExpr, args: &[ExprRef]) -> SQLPlannerResult unsupported_sql_err!("map_groups"), } } + +mod static_docs { + pub(crate) const COUNT_DOCSTRING: &str = + "Counts the number of non-null elements in the input expression. + +Example: + +.. code-block:: sql + :caption: SQL + + SELECT count(x) FROM tbl + +.. code-block:: text + :caption: Input + + ╭───────╮ + │ x │ + │ --- │ + │ Int64 │ + ╞═══════╡ + │ 100 │ + ├╌╌╌╌╌╌╌┤ + │ 200 │ + ├╌╌╌╌╌╌╌┤ + │ null │ + ╰───────╯ + (Showing first 3 of 3 rows) + +.. code-block:: text + :caption: Output + + ╭───────╮ + │ x │ + │ --- │ + │ Int64 │ + ╞═══════╡ + │ 1 │ + ╰───────╯ + (Showing first 1 of 1 rows)"; + + pub(crate) const SUM_DOCSTRING: &str = + "Calculates the sum of non-null elements in the input expression. + +Example: + +.. code-block:: sql + :caption: SQL + + SELECT sum(x) FROM tbl + +.. code-block:: text + :caption: Input + + ╭───────╮ + │ x │ + │ --- │ + │ Int64 │ + ╞═══════╡ + │ 100 │ + ├╌╌╌╌╌╌╌┤ + │ 200 │ + ├╌╌╌╌╌╌╌┤ + │ null │ + ╰───────╯ + (Showing first 3 of 3 rows) + +.. code-block:: text + :caption: Output + + ╭───────╮ + │ x │ + │ --- │ + │ Int64 │ + ╞═══════╡ + │ 300 │ + ╰───────╯ + (Showing first 1 of 1 rows)"; + + pub(crate) const AVG_DOCSTRING: &str = + "Calculates the average (mean) of non-null elements in the input expression. + +.. seealso:: + This SQL Function has aliases. + + * :func:`~daft.sql._sql_funcs.mean` + * :func:`~daft.sql._sql_funcs.avg` + +Example: + +.. code-block:: sql + :caption: SQL + + SELECT {}(x) FROM tbl + +.. code-block:: text + :caption: Input + + ╭───────╮ + │ x │ + │ --- │ + │ Int64 │ + ╞═══════╡ + │ 100 │ + ├╌╌╌╌╌╌╌┤ + │ 200 │ + ├╌╌╌╌╌╌╌┤ + │ null │ + ╰───────╯ + (Showing first 3 of 3 rows) + +.. code-block:: text + :caption: Output + + ╭───────────╮ + │ x │ + │ --- │ + │ Float64 │ + ╞═══════════╡ + │ 150.0 │ + ╰───────────╯ + (Showing first 1 of 1 rows)"; + + pub(crate) const MIN_DOCSTRING: &str = + "Finds the minimum value among non-null elements in the input expression. + +Example: + +.. code-block:: sql + :caption: SQL + + SELECT min(x) FROM tbl + +.. code-block:: text + :caption: Input + + ╭───────╮ + │ x │ + │ --- │ + │ Int64 │ + ╞═══════╡ + │ 100 │ + ├╌╌╌╌╌╌╌┤ + │ 200 │ + ├╌╌╌╌╌╌╌┤ + │ null │ + ╰───────╯ + (Showing first 3 of 3 rows) + +.. code-block:: text + :caption: Output + + ╭───────╮ + │ x │ + │ --- │ + │ Int64 │ + ╞═══════╡ + │ 100 │ + ╰───────╯ + (Showing first 1 of 1 rows)"; + + pub(crate) const MAX_DOCSTRING: &str = + "Finds the maximum value among non-null elements in the input expression. + +Example: + +.. code-block:: sql + :caption: SQL + + SELECT max(x) FROM tbl + +.. code-block:: text + :caption: Input + + ╭───────╮ + │ x │ + │ --- │ + │ Int64 │ + ╞═══════╡ + │ 100 │ + ├╌╌╌╌╌╌╌┤ + │ 200 │ + ├╌╌╌╌╌╌╌┤ + │ null │ + ╰───────╯ + (Showing first 3 of 3 rows) + +.. code-block:: text + :caption: Output + + ╭───────╮ + │ x │ + │ --- │ + │ Int64 │ + ╞═══════╡ + │ 200 │ + ╰───────╯ + (Showing first 1 of 1 rows)"; +} diff --git a/src/daft-sql/src/modules/float.rs b/src/daft-sql/src/modules/float.rs index 4cfffe34b4..292a5c4d85 100644 --- a/src/daft-sql/src/modules/float.rs +++ b/src/daft-sql/src/modules/float.rs @@ -37,6 +37,14 @@ impl SQLFunction for SQLFillNan { _ => unsupported_sql_err!("Invalid arguments for 'fill_nan': '{inputs:?}'"), } } + + fn docstrings(&self, _alias: &str) -> String { + static_docs::FILL_NAN_DOCSTRING.to_string() + } + + fn arg_names(&self) -> &'static [&'static str] { + &["input", "fill_value"] + } } pub struct SQLIsInf {} @@ -52,6 +60,14 @@ impl SQLFunction for SQLIsInf { _ => unsupported_sql_err!("Invalid arguments for 'is_inf': '{inputs:?}'"), } } + + fn docstrings(&self, _alias: &str) -> String { + static_docs::IS_INF_DOCSTRING.to_string() + } + + fn arg_names(&self) -> &'static [&'static str] { + &["input"] + } } pub struct SQLIsNan {} @@ -67,6 +83,14 @@ impl SQLFunction for SQLIsNan { _ => unsupported_sql_err!("Invalid arguments for 'is_nan': '{inputs:?}'"), } } + + fn docstrings(&self, _alias: &str) -> String { + static_docs::IS_NAN_DOCSTRING.to_string() + } + + fn arg_names(&self) -> &'static [&'static str] { + &["input"] + } } pub struct SQLNotNan {} @@ -82,4 +106,26 @@ impl SQLFunction for SQLNotNan { _ => unsupported_sql_err!("Invalid arguments for 'not_nan': '{inputs:?}'"), } } + + fn docstrings(&self, _alias: &str) -> String { + static_docs::NOT_NAN_DOCSTRING.to_string() + } + + fn arg_names(&self) -> &'static [&'static str] { + &["input"] + } +} + +mod static_docs { + pub(crate) const FILL_NAN_DOCSTRING: &str = + "Replaces NaN values in the input expression with a specified fill value."; + + pub(crate) const IS_INF_DOCSTRING: &str = + "Checks if the input expression is infinite (positive or negative infinity)."; + + pub(crate) const IS_NAN_DOCSTRING: &str = + "Checks if the input expression is NaN (Not a Number)."; + + pub(crate) const NOT_NAN_DOCSTRING: &str = + "Checks if the input expression is not NaN (Not a Number)."; } diff --git a/src/daft-sql/src/modules/image/crop.rs b/src/daft-sql/src/modules/image/crop.rs index 36c72fcca3..286208889c 100644 --- a/src/daft-sql/src/modules/image/crop.rs +++ b/src/daft-sql/src/modules/image/crop.rs @@ -21,4 +21,12 @@ impl SQLFunction for SQLImageCrop { _ => unsupported_sql_err!("Invalid arguments for image_crop: '{inputs:?}'"), } } + + fn docstrings(&self, _alias: &str) -> String { + "Crops an image to a specified bounding box. The bounding box is specified as [x, y, width, height].".to_string() + } + + fn arg_names(&self) -> &'static [&'static str] { + &["input_image", "bounding_box"] + } } diff --git a/src/daft-sql/src/modules/image/decode.rs b/src/daft-sql/src/modules/image/decode.rs index a6b95d538d..a896c67a05 100644 --- a/src/daft-sql/src/modules/image/decode.rs +++ b/src/daft-sql/src/modules/image/decode.rs @@ -61,4 +61,12 @@ impl SQLFunction for SQLImageDecode { _ => unsupported_sql_err!("Invalid arguments for image_decode: '{inputs:?}'"), } } + + fn docstrings(&self, _alias: &str) -> String { + "Decodes an image from binary data. Optionally, you can specify the image mode and error handling behavior.".to_string() + } + + fn arg_names(&self) -> &'static [&'static str] { + &["input", "mode", "on_error"] + } } diff --git a/src/daft-sql/src/modules/image/encode.rs b/src/daft-sql/src/modules/image/encode.rs index a902179f88..acf489c807 100644 --- a/src/daft-sql/src/modules/image/encode.rs +++ b/src/daft-sql/src/modules/image/encode.rs @@ -46,4 +46,12 @@ impl SQLFunction for SQLImageEncode { _ => unsupported_sql_err!("Invalid arguments for image_encode: '{inputs:?}'"), } } + + fn docstrings(&self, _alias: &str) -> String { + "Encodes an image into the specified image file format, returning a binary column of encoded bytes.".to_string() + } + + fn arg_names(&self) -> &'static [&'static str] { + &["input_image", "image_format"] + } } diff --git a/src/daft-sql/src/modules/image/resize.rs b/src/daft-sql/src/modules/image/resize.rs index 8ce37eb7f8..e4c9804d39 100644 --- a/src/daft-sql/src/modules/image/resize.rs +++ b/src/daft-sql/src/modules/image/resize.rs @@ -64,4 +64,12 @@ impl SQLFunction for SQLImageResize { _ => unsupported_sql_err!("Invalid arguments for image_resize: '{inputs:?}'"), } } + + fn docstrings(&self, _alias: &str) -> String { + "Resizes an image to the specified width and height.".to_string() + } + + fn arg_names(&self) -> &'static [&'static str] { + &["input_image", "width", "height"] + } } diff --git a/src/daft-sql/src/modules/image/to_mode.rs b/src/daft-sql/src/modules/image/to_mode.rs index a02efb2d36..b5b9202d1f 100644 --- a/src/daft-sql/src/modules/image/to_mode.rs +++ b/src/daft-sql/src/modules/image/to_mode.rs @@ -41,4 +41,12 @@ impl SQLFunction for SQLImageToMode { _ => unsupported_sql_err!("Invalid arguments for image_encode: '{inputs:?}'"), } } + + fn docstrings(&self, _alias: &str) -> String { + "Converts an image to the specified mode (e.g. RGB, RGBA, Grayscale).".to_string() + } + + fn arg_names(&self) -> &'static [&'static str] { + &["input_image", "mode"] + } } diff --git a/src/daft-sql/src/modules/json.rs b/src/daft-sql/src/modules/json.rs index f0d600daea..8dc9e617f5 100644 --- a/src/daft-sql/src/modules/json.rs +++ b/src/daft-sql/src/modules/json.rs @@ -35,4 +35,17 @@ impl SQLFunction for JsonQuery { ), } } + + fn docstrings(&self, _alias: &str) -> String { + static_docs::JSON_QUERY_DOCSTRING.to_string() + } + + fn arg_names(&self) -> &'static [&'static str] { + &["input", "query"] + } +} + +mod static_docs { + pub(crate) const JSON_QUERY_DOCSTRING: &str = + "Extracts a JSON object from a JSON string using a JSONPath expression."; } diff --git a/src/daft-sql/src/modules/list.rs b/src/daft-sql/src/modules/list.rs index b9e52d9748..bd6db25990 100644 --- a/src/daft-sql/src/modules/list.rs +++ b/src/daft-sql/src/modules/list.rs @@ -55,6 +55,14 @@ impl SQLFunction for SQLListChunk { ), } } + + fn docstrings(&self, _alias: &str) -> String { + static_docs::LIST_CHUNK_DOCSTRING.to_string() + } + + fn arg_names(&self) -> &'static [&'static str] { + &["input", "chunk_size"] + } } pub struct SQLListCount; @@ -86,6 +94,14 @@ impl SQLFunction for SQLListCount { _ => unsupported_sql_err!("invalid arguments for list_count. Expected either list_count(expr) or list_count(expr, mode)"), } } + + fn docstrings(&self, _alias: &str) -> String { + static_docs::LIST_COUNT_DOCSTRING.to_string() + } + + fn arg_names(&self) -> &'static [&'static str] { + &["input", "mode"] + } } pub struct SQLExplode; @@ -104,6 +120,14 @@ impl SQLFunction for SQLExplode { _ => unsupported_sql_err!("Expected 1 argument"), } } + + fn docstrings(&self, _alias: &str) -> String { + static_docs::EXPLODE_DOCSTRING.to_string() + } + + fn arg_names(&self) -> &'static [&'static str] { + &["input"] + } } pub struct SQLListJoin; @@ -125,6 +149,14 @@ impl SQLFunction for SQLListJoin { ), } } + + fn docstrings(&self, _alias: &str) -> String { + static_docs::LIST_JOIN_DOCSTRING.to_string() + } + + fn arg_names(&self) -> &'static [&'static str] { + &["input", "separator"] + } } pub struct SQLListMax; @@ -143,6 +175,14 @@ impl SQLFunction for SQLListMax { _ => unsupported_sql_err!("invalid arguments for list_max. Expected list_max(expr)"), } } + + fn docstrings(&self, _alias: &str) -> String { + static_docs::LIST_MAX_DOCSTRING.to_string() + } + + fn arg_names(&self) -> &'static [&'static str] { + &["input"] + } } pub struct SQLListMean; @@ -161,6 +201,14 @@ impl SQLFunction for SQLListMean { _ => unsupported_sql_err!("invalid arguments for list_mean. Expected list_mean(expr)"), } } + + fn docstrings(&self, _alias: &str) -> String { + static_docs::LIST_MEAN_DOCSTRING.to_string() + } + + fn arg_names(&self) -> &'static [&'static str] { + &["input"] + } } pub struct SQLListMin; @@ -179,6 +227,14 @@ impl SQLFunction for SQLListMin { _ => unsupported_sql_err!("invalid arguments for list_min. Expected list_min(expr)"), } } + + fn docstrings(&self, _alias: &str) -> String { + static_docs::LIST_MIN_DOCSTRING.to_string() + } + + fn arg_names(&self) -> &'static [&'static str] { + &["input"] + } } pub struct SQLListSum; @@ -197,6 +253,14 @@ impl SQLFunction for SQLListSum { _ => unsupported_sql_err!("invalid arguments for list_sum. Expected list_sum(expr)"), } } + + fn docstrings(&self, _alias: &str) -> String { + static_docs::LIST_SUM_DOCSTRING.to_string() + } + + fn arg_names(&self) -> &'static [&'static str] { + &["input"] + } } pub struct SQLListSlice; @@ -219,6 +283,14 @@ impl SQLFunction for SQLListSlice { ), } } + + fn docstrings(&self, _alias: &str) -> String { + static_docs::LIST_SLICE_DOCSTRING.to_string() + } + + fn arg_names(&self) -> &'static [&'static str] { + &["input", "start", "end"] + } } pub struct SQLListSort; @@ -258,4 +330,38 @@ impl SQLFunction for SQLListSort { ), } } + + fn docstrings(&self, _alias: &str) -> String { + static_docs::LIST_SORT_DOCSTRING.to_string() + } + + fn arg_names(&self) -> &'static [&'static str] { + &["input", "order"] + } +} + +mod static_docs { + pub(crate) const LIST_CHUNK_DOCSTRING: &str = "Splits a list into chunks of a specified size."; + + pub(crate) const LIST_COUNT_DOCSTRING: &str = "Counts the number of elements in a list."; + + pub(crate) const EXPLODE_DOCSTRING: &str = "Expands a list column into multiple rows."; + + pub(crate) const LIST_JOIN_DOCSTRING: &str = + "Joins elements of a list into a single string using a specified separator."; + + pub(crate) const LIST_MAX_DOCSTRING: &str = "Returns the maximum value in a list."; + + pub(crate) const LIST_MEAN_DOCSTRING: &str = + "Calculates the mean (average) of values in a list."; + + pub(crate) const LIST_MIN_DOCSTRING: &str = "Returns the minimum value in a list."; + + pub(crate) const LIST_SUM_DOCSTRING: &str = "Calculates the sum of values in a list."; + + pub(crate) const LIST_SLICE_DOCSTRING: &str = + "Extracts a portion of a list from a start index to an end index."; + + pub(crate) const LIST_SORT_DOCSTRING: &str = + "Sorts the elements of a list in ascending or descending order."; } diff --git a/src/daft-sql/src/modules/map.rs b/src/daft-sql/src/modules/map.rs index d3a328f3a4..0ae5aca2be 100644 --- a/src/daft-sql/src/modules/map.rs +++ b/src/daft-sql/src/modules/map.rs @@ -30,4 +30,23 @@ impl SQLFunction for MapGet { _ => invalid_operation_err!("Expected 2 input args"), } } + + fn docstrings(&self, alias: &str) -> String { + static_docs::MAP_GET_DOCSTRING.replace("{}", alias) + } + + fn arg_names(&self) -> &'static [&'static str] { + &["input", "key"] + } +} + +mod static_docs { + pub(crate) const MAP_GET_DOCSTRING: &str = + "Retrieves the value associated with a given key from a map. + +.. seealso:: + + * :func:`~daft.sql._sql_funcs.map_get` + * :func:`~daft.sql._sql_funcs.map_extract` +"; } diff --git a/src/daft-sql/src/modules/numeric.rs b/src/daft-sql/src/modules/numeric.rs index 197d958860..21ac2a0873 100644 --- a/src/daft-sql/src/modules/numeric.rs +++ b/src/daft-sql/src/modules/numeric.rs @@ -88,6 +88,67 @@ impl SQLFunction for SQLNumericExpr { let inputs = self.args_to_expr_unnamed(inputs, planner)?; to_expr(self, inputs.as_slice()) } + + fn docstrings(&self, _alias: &str) -> String { + let docstring = match self { + Self::Abs => "Gets the absolute value of a number.", + Self::Ceil => "Rounds a number up to the nearest integer.", + Self::Exp => "Calculates the exponential of a number (e^x).", + Self::Floor => "Rounds a number down to the nearest integer.", + Self::Round => "Rounds a number to a specified number of decimal places.", + Self::Sign => "Returns the sign of a number (-1, 0, or 1).", + Self::Sqrt => "Calculates the square root of a number.", + Self::Sin => "Calculates the sine of an angle in radians.", + Self::Cos => "Calculates the cosine of an angle in radians.", + Self::Tan => "Calculates the tangent of an angle in radians.", + Self::Cot => "Calculates the cotangent of an angle in radians.", + Self::ArcSin => "Calculates the inverse sine (arc sine) of a number.", + Self::ArcCos => "Calculates the inverse cosine (arc cosine) of a number.", + Self::ArcTan => "Calculates the inverse tangent (arc tangent) of a number.", + Self::ArcTan2 => { + "Calculates the angle between the positive x-axis and the ray from (0,0) to (x,y)." + } + Self::Radians => "Converts an angle from degrees to radians.", + Self::Degrees => "Converts an angle from radians to degrees.", + Self::Log => "Calculates the natural logarithm of a number.", + Self::Log2 => "Calculates the base-2 logarithm of a number.", + Self::Log10 => "Calculates the base-10 logarithm of a number.", + Self::Ln => "Calculates the natural logarithm of a number.", + Self::ArcTanh => "Calculates the inverse hyperbolic tangent of a number.", + Self::ArcCosh => "Calculates the inverse hyperbolic cosine of a number.", + Self::ArcSinh => "Calculates the inverse hyperbolic sine of a number.", + }; + docstring.to_string() + } + + fn arg_names(&self) -> &'static [&'static str] { + match self { + Self::Abs + | Self::Ceil + | Self::Floor + | Self::Sign + | Self::Sqrt + | Self::Sin + | Self::Cos + | Self::Tan + | Self::Cot + | Self::ArcSin + | Self::ArcCos + | Self::ArcTan + | Self::Radians + | Self::Degrees + | Self::Log2 + | Self::Log10 + | Self::Ln + | Self::ArcTanh + | Self::ArcCosh + | Self::ArcSinh => &["input"], + Self::Log => &["input", "base"], + Self::Round => &["input", "precision"], + Self::Exp => &["input", "exponent"], + Self::ArcTan2 => &["y", "x"], + } + } } fn to_expr(expr: &SQLNumericExpr, args: &[ExprRef]) -> SQLPlannerResult { diff --git a/src/daft-sql/src/modules/partitioning.rs b/src/daft-sql/src/modules/partitioning.rs index e833edd51d..def20b2774 100644 --- a/src/daft-sql/src/modules/partitioning.rs +++ b/src/daft-sql/src/modules/partitioning.rs @@ -80,6 +80,28 @@ impl SQLFunction for PartitioningExpr { } } } + + fn docstrings(&self, _alias: &str) -> String { + match self { + Self::Years => "Extracts the number of years since epoch time from a datetime expression.".to_string(), + Self::Months => "Extracts the number of months since epoch time from a datetime expression.".to_string(), + Self::Days => "Extracts the number of days since epoch time from a datetime expression.".to_string(), + Self::Hours => "Extracts the number of hours since epoch time from a datetime expression.".to_string(), + Self::IcebergBucket(_) => "Computes a bucket number for the input expression based the specified number of buckets using an Iceberg-specific hash.".to_string(), + Self::IcebergTruncate(_) => "Truncates the input expression to a specified width.".to_string(), + } + } + + fn arg_names(&self) -> &'static [&'static str] { + match self { + Self::Years => &["input"], + Self::Months => &["input"], + Self::Days => &["input"], + Self::Hours => &["input"], + Self::IcebergBucket(_) => &["input", "num_buckets"], + Self::IcebergTruncate(_) => &["input", "width"], + } + } } fn partitioning_helper daft_dsl::ExprRef>( diff --git a/src/daft-sql/src/modules/structs.rs b/src/daft-sql/src/modules/structs.rs index 66be42d8e3..17fae85c9e 100644 --- a/src/daft-sql/src/modules/structs.rs +++ b/src/daft-sql/src/modules/structs.rs @@ -34,4 +34,12 @@ impl SQLFunction for StructGet { _ => invalid_operation_err!("Expected 2 input args"), } } + + fn docstrings(&self, _alias: &str) -> String { + "Extracts a field from a struct expression by name.".to_string() + } + + fn arg_names(&self) -> &'static [&'static str] { + &["input", "field"] + } } diff --git a/src/daft-sql/src/modules/temporal.rs b/src/daft-sql/src/modules/temporal.rs index 58687724fa..840c278765 100644 --- a/src/daft-sql/src/modules/temporal.rs +++ b/src/daft-sql/src/modules/temporal.rs @@ -50,6 +50,16 @@ macro_rules! temporal { ), } } + fn docstrings(&self, _alias: &str) -> String { + format!( + "Extracts the {} component from a datetime expression.", + stringify!($fn_name).replace("dt_", "") + ) + } + + fn arg_names(&self) -> &'static [&'static str] { + &["input"] + } } }; } diff --git a/src/daft-sql/src/modules/utf8.rs b/src/daft-sql/src/modules/utf8.rs index 6e7b6b68a5..c31879cd82 100644 --- a/src/daft-sql/src/modules/utf8.rs +++ b/src/daft-sql/src/modules/utf8.rs @@ -72,6 +72,72 @@ impl SQLFunction for Utf8Expr { let inputs = self.args_to_expr_unnamed(inputs, planner)?; to_expr(self, &inputs) } + + fn docstrings(&self, _alias: &str) -> String { + match self { + Self::EndsWith => "Returns true if the string ends with the specified substring".to_string(), + Self::StartsWith => "Returns true if the string starts with the specified substring".to_string(), + Self::Contains => "Returns true if the string contains the specified substring".to_string(), + Self::Split(_) => "Splits the string by the specified delimiter and returns an array of substrings".to_string(), + Self::Match => "Returns true if the string matches the specified regular expression pattern".to_string(), + Self::Extract(_) => "Extracts the first substring that matches the specified regular expression pattern".to_string(), + Self::ExtractAll(_) => "Extracts all substrings that match the specified regular expression pattern".to_string(), + Self::Replace(_) => "Replaces all occurrences of a substring with a new string".to_string(), + Self::Like => "Returns true if the string matches the specified SQL LIKE pattern".to_string(), + Self::Ilike => "Returns true if the string matches the specified SQL LIKE pattern (case-insensitive)".to_string(), + Self::Length => "Returns the length of the string".to_string(), + Self::Lower => "Converts the string to lowercase".to_string(), + Self::Upper => "Converts the string to uppercase".to_string(), + Self::Lstrip => "Removes leading whitespace from the string".to_string(), + Self::Rstrip => "Removes trailing whitespace from the string".to_string(), + Self::Reverse => "Reverses the order of characters in the string".to_string(), + Self::Capitalize => "Capitalizes the first character of the string".to_string(), + Self::Left => "Returns the specified number of leftmost characters from the string".to_string(), + Self::Right => "Returns the specified number of rightmost characters from the string".to_string(), + Self::Find => "Returns the index of the first occurrence of a substring within the string".to_string(), + Self::Rpad => "Pads the string on the right side with the specified string until it reaches the specified length".to_string(), + Self::Lpad => "Pads the string on the left side with the specified string until it reaches the specified length".to_string(), + Self::Repeat => "Repeats the string the specified number of times".to_string(), + Self::Substr => "Returns a substring of the string starting at the specified position and length".to_string(), + Self::ToDate(_) => "Parses the string as a date using the specified format.".to_string(), + Self::ToDatetime(_, _) => "Parses the string as a datetime using the specified format.".to_string(), + Self::LengthBytes => "Returns the length of the string in bytes".to_string(), + Self::Normalize(_) => unimplemented!("Normalize not implemented"), + } + } + + fn arg_names(&self) -> &'static [&'static str] { + match self { + Self::EndsWith => &["string_input", "substring"], + Self::StartsWith => &["string_input", "substring"], + Self::Contains => &["string_input", "substring"], + Self::Split(_) => &["string_input", "delimiter"], + Self::Match => &["string_input", "pattern"], + Self::Extract(_) => &["string_input", "pattern"], + Self::ExtractAll(_) => &["string_input", "pattern"], + Self::Replace(_) => &["string_input", "pattern", "replacement"], + Self::Like => &["string_input", "pattern"], + Self::Ilike => &["string_input", "pattern"], + Self::Length => &["string_input"], + Self::Lower => &["string_input"], + Self::Upper => &["string_input"], + Self::Lstrip => &["string_input"], + Self::Rstrip => &["string_input"], + Self::Reverse => &["string_input"], + Self::Capitalize => &["string_input"], + Self::Left => &["string_input", "length"], + Self::Right => &["string_input", "length"], + Self::Find => &["string_input", "substring"], + Self::Rpad => &["string_input", "length", "pad"], + Self::Lpad => &["string_input", "length", "pad"], + Self::Repeat => &["string_input", "count"], + Self::Substr => &["string_input", "start", "length"], + Self::ToDate(_) => &["string_input", "format"], + Self::ToDatetime(_, _) => &["string_input", "format"], + Self::LengthBytes => &["string_input"], + Self::Normalize(_) => unimplemented!("Normalize not implemented"), + } + } } fn to_expr(expr: &Utf8Expr, args: &[ExprRef]) -> SQLPlannerResult { diff --git a/src/daft-sql/src/python.rs b/src/daft-sql/src/python.rs index 283184f014..b61d3fedd2 100644 --- a/src/daft-sql/src/python.rs +++ b/src/daft-sql/src/python.rs @@ -5,6 +5,31 @@ use pyo3::prelude::*; use crate::{catalog::SQLCatalog, functions::SQL_FUNCTIONS, planner::SQLPlanner}; +#[pyclass] +pub struct SQLFunctionStub { + name: String, + docstring: String, + arg_names: Vec<&'static str>, +} + +#[pymethods] +impl SQLFunctionStub { + #[getter] + fn name(&self) -> PyResult { + Ok(self.name.clone()) + } + + #[getter] + fn docstring(&self) -> PyResult { + Ok(self.docstring.clone()) + } + + #[getter] + fn arg_names(&self) -> PyResult> { + Ok(self.arg_names.clone()) + } +} + #[pyfunction] pub fn sql( sql: &str, @@ -23,8 +48,20 @@ pub fn sql_expr(sql: &str) -> PyResult { } #[pyfunction] -pub fn list_sql_functions() -> Vec { - SQL_FUNCTIONS.map.keys().cloned().collect() +pub fn list_sql_functions() -> Vec { + SQL_FUNCTIONS + .map + .keys() + .cloned() + .map(|name| { + let (docstring, args) = SQL_FUNCTIONS.docsmap.get(&name).unwrap(); + SQLFunctionStub { + name, + docstring: docstring.to_string(), + arg_names: args.to_vec(), + } + }) + .collect() } /// PyCatalog is the Python interface to the Catalog. From fe4553f533da0838494ff215ac139627cbf2426c Mon Sep 17 00:00:00 2001 From: Cory Grinstead Date: Tue, 1 Oct 2024 12:23:14 -0500 Subject: [PATCH 3/5] [FEAT]: sql `read_deltalake` function (#2974) depends on https://github.com/Eventual-Inc/Daft/pull/2954 --------- Co-authored-by: Kev Wang --- Cargo.lock | 1 + .../file-formats/src/file_format_config.rs | 11 + src/daft-dsl/src/lib.rs | 2 +- src/daft-dsl/src/lit.rs | 75 +++- src/daft-plan/src/builder.rs | 179 +++++++- src/daft-plan/src/lib.rs | 2 +- src/daft-scan/src/lib.rs | 2 +- src/daft-schema/src/time_unit.rs | 16 + src/daft-sql/Cargo.toml | 1 + src/daft-sql/src/error.rs | 9 + src/daft-sql/src/functions.rs | 35 +- src/daft-sql/src/lib.rs | 2 +- src/daft-sql/src/modules/config.rs | 391 ++++++++++++++++++ src/daft-sql/src/modules/image/resize.rs | 4 +- src/daft-sql/src/modules/mod.rs | 1 + src/daft-sql/src/planner.rs | 31 +- src/daft-sql/src/table_provider/mod.rs | 119 ++++++ .../src/table_provider/read_parquet.rs | 77 ++++ tests/sql/test_table_funcs.py | 7 + 19 files changed, 933 insertions(+), 32 deletions(-) create mode 100644 src/daft-sql/src/modules/config.rs create mode 100644 src/daft-sql/src/table_provider/mod.rs create mode 100644 src/daft-sql/src/table_provider/read_parquet.rs create mode 100644 tests/sql/test_table_funcs.py diff --git a/Cargo.lock b/Cargo.lock index 592a0793ba..8d60981bd3 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2170,6 +2170,7 @@ version = "0.3.0-dev0" dependencies = [ "common-daft-config", "common-error", + "common-io-config", "daft-core", "daft-dsl", "daft-functions", diff --git a/src/common/file-formats/src/file_format_config.rs b/src/common/file-formats/src/file_format_config.rs index fe659bc444..6054907861 100644 --- a/src/common/file-formats/src/file_format_config.rs +++ b/src/common/file-formats/src/file_format_config.rs @@ -115,6 +115,17 @@ impl ParquetSourceConfig { } } +impl Default for ParquetSourceConfig { + fn default() -> Self { + Self { + coerce_int96_timestamp_unit: TimeUnit::Nanoseconds, + field_id_mapping: None, + row_groups: None, + chunk_size: None, + } + } +} + #[cfg(feature = "python")] #[pymethods] impl ParquetSourceConfig { diff --git a/src/daft-dsl/src/lib.rs b/src/daft-dsl/src/lib.rs index 754578eb6d..2fa99115e3 100644 --- a/src/daft-dsl/src/lib.rs +++ b/src/daft-dsl/src/lib.rs @@ -18,7 +18,7 @@ pub use expr::{ binary_op, col, has_agg, has_stateful_udf, is_partition_compatible, AggExpr, ApproxPercentileParams, Expr, ExprRef, Operator, SketchType, }; -pub use lit::{lit, literals_to_series, null_lit, Literal, LiteralValue}; +pub use lit::{lit, literal_value, literals_to_series, null_lit, Literal, LiteralValue}; #[cfg(feature = "python")] use pyo3::prelude::*; pub use resolve_expr::{ diff --git a/src/daft-dsl/src/lit.rs b/src/daft-dsl/src/lit.rs index 5db0f05a3d..55888d73f8 100644 --- a/src/daft-dsl/src/lit.rs +++ b/src/daft-dsl/src/lit.rs @@ -14,6 +14,7 @@ use daft_core::{ display_timestamp, }, }; +use indexmap::IndexMap; use serde::{Deserialize, Serialize}; #[cfg(feature = "python")] @@ -68,6 +69,8 @@ pub enum LiteralValue { /// Python object. #[cfg(feature = "python")] Python(PyObjectWrapper), + + Struct(IndexMap), } impl Eq for LiteralValue {} @@ -112,6 +115,12 @@ impl Hash for LiteralValue { } #[cfg(feature = "python")] Python(py_obj) => py_obj.hash(state), + Struct(entries) => { + entries.iter().for_each(|(v, f)| { + v.hash(state); + f.hash(state); + }); + } } } } @@ -143,6 +152,16 @@ impl Display for LiteralValue { Python::with_gil(|py| pyobj.0.call_method0(py, pyo3::intern!(py, "__str__"))) .unwrap() }), + Struct(entries) => { + write!(f, "Struct(")?; + for (i, (field, v)) in entries.iter().enumerate() { + if i > 0 { + write!(f, ", ")?; + } + write!(f, "{}: {}", field.name, v)?; + } + write!(f, ")") + } } } } @@ -169,6 +188,7 @@ impl LiteralValue { Series(series) => series.data_type().clone(), #[cfg(feature = "python")] Python(_) => DataType::Python, + Struct(entries) => DataType::Struct(entries.keys().cloned().collect()), } } @@ -203,6 +223,13 @@ impl LiteralValue { Series(series) => series.clone().rename("literal"), #[cfg(feature = "python")] Python(val) => PythonArray::from(("literal", vec![val.0.clone()])).into_series(), + Struct(entries) => { + let struct_dtype = DataType::Struct(entries.keys().cloned().collect()); + let struct_field = Field::new("literal", struct_dtype); + + let values = entries.values().map(|v| v.to_series()).collect(); + StructArray::new(struct_field, values, None).into_series() + } }; result } @@ -235,6 +262,7 @@ impl LiteralValue { Decimal(..) | Series(..) | Time(..) | Binary(..) => display_sql_err, #[cfg(feature = "python")] Python(..) => display_sql_err, + Struct(..) => display_sql_err, } } @@ -304,49 +332,64 @@ impl LiteralValue { } } -pub trait Literal { +pub trait Literal: Sized { /// [Literal](Expr::Literal) expression. - fn lit(self) -> ExprRef; + fn lit(self) -> ExprRef { + Expr::Literal(self.literal_value()).into() + } + fn literal_value(self) -> LiteralValue; } impl Literal for String { - fn lit(self) -> ExprRef { - Expr::Literal(LiteralValue::Utf8(self)).into() + fn literal_value(self) -> LiteralValue { + LiteralValue::Utf8(self) } } impl<'a> Literal for &'a str { - fn lit(self) -> ExprRef { - Expr::Literal(LiteralValue::Utf8(self.to_owned())).into() + fn literal_value(self) -> LiteralValue { + LiteralValue::Utf8(self.to_owned()) } } macro_rules! make_literal { ($TYPE:ty, $SCALAR:ident) => { impl Literal for $TYPE { - fn lit(self) -> ExprRef { - Expr::Literal(LiteralValue::$SCALAR(self)).into() + fn literal_value(self) -> LiteralValue { + LiteralValue::$SCALAR(self) } } }; } impl<'a> Literal for &'a [u8] { - fn lit(self) -> ExprRef { - Expr::Literal(LiteralValue::Binary(self.to_vec())).into() + fn literal_value(self) -> LiteralValue { + LiteralValue::Binary(self.to_vec()) } } impl Literal for Series { - fn lit(self) -> ExprRef { - Expr::Literal(LiteralValue::Series(self)).into() + fn literal_value(self) -> LiteralValue { + LiteralValue::Series(self) } } #[cfg(feature = "python")] impl Literal for pyo3::PyObject { - fn lit(self) -> ExprRef { - Expr::Literal(LiteralValue::Python(PyObjectWrapper(self))).into() + fn literal_value(self) -> LiteralValue { + LiteralValue::Python(PyObjectWrapper(self)) + } +} + +impl Literal for Option +where + T: Literal, +{ + fn literal_value(self) -> LiteralValue { + match self { + Some(val) => val.literal_value(), + None => LiteralValue::Null, + } } } @@ -361,6 +404,10 @@ pub fn lit(t: L) -> ExprRef { t.lit() } +pub fn literal_value(t: L) -> LiteralValue { + t.literal_value() +} + pub fn null_lit() -> ExprRef { Arc::new(Expr::Literal(LiteralValue::Null)) } diff --git a/src/daft-plan/src/builder.rs b/src/daft-plan/src/builder.rs index 982a3634a9..a9e05ec6cb 100644 --- a/src/daft-plan/src/builder.rs +++ b/src/daft-plan/src/builder.rs @@ -1,17 +1,27 @@ use std::{ - collections::{HashMap, HashSet}, + collections::{BTreeMap, HashMap, HashSet}, sync::Arc, }; use common_daft_config::DaftPlanningConfig; use common_display::mermaid::MermaidDisplayOptions; use common_error::DaftResult; -use common_file_formats::FileFormat; +use common_file_formats::{FileFormat, FileFormatConfig, ParquetSourceConfig}; use common_io_config::IOConfig; -use daft_core::join::{JoinStrategy, JoinType}; +use daft_core::{ + join::{JoinStrategy, JoinType}, + prelude::TimeUnit, +}; use daft_dsl::{col, ExprRef}; -use daft_scan::{PhysicalScanInfo, Pushdowns, ScanOperatorRef}; -use daft_schema::schema::{Schema, SchemaRef}; +use daft_scan::{ + glob::GlobScanOperator, + storage_config::{NativeStorageConfig, StorageConfig}, + PhysicalScanInfo, Pushdowns, ScanOperatorRef, +}; +use daft_schema::{ + field::Field, + schema::{Schema, SchemaRef}, +}; #[cfg(feature = "python")] use { crate::sink_info::{CatalogInfo, IcebergCatalogInfo}, @@ -73,7 +83,29 @@ impl From<&LogicalPlanBuilder> for LogicalPlanRef { value.plan.clone() } } - +pub trait IntoGlobPath { + fn into_glob_path(self) -> Vec; +} +impl IntoGlobPath for Vec { + fn into_glob_path(self) -> Vec { + self + } +} +impl IntoGlobPath for String { + fn into_glob_path(self) -> Vec { + vec![self] + } +} +impl IntoGlobPath for &str { + fn into_glob_path(self) -> Vec { + vec![self.to_string()] + } +} +impl IntoGlobPath for Vec<&str> { + fn into_glob_path(self) -> Vec { + self.iter().map(|s| s.to_string()).collect() + } +} impl LogicalPlanBuilder { /// Replace the LogicalPlanBuilder's plan with the provided plan pub fn with_new_plan>>(&self, plan: LP) -> Self { @@ -105,9 +137,51 @@ impl LogicalPlanBuilder { )); let logical_plan: LogicalPlan = logical_ops::Source::new(schema.clone(), source_info.into()).into(); + Ok(Self::new(logical_plan.into(), None)) } + #[cfg(feature = "python")] + pub fn delta_scan>( + glob_path: T, + io_config: Option, + multithreaded_io: bool, + ) -> DaftResult { + use daft_scan::storage_config::PyStorageConfig; + + Python::with_gil(|py| { + let io_config = io_config.unwrap_or_default(); + + let native_storage_config = NativeStorageConfig { + io_config: Some(io_config), + multithreaded_io, + }; + + let py_storage_config: PyStorageConfig = + Arc::new(StorageConfig::Native(Arc::new(native_storage_config))).into(); + + // let py_io_config = PyIOConfig { config: io_config }; + let delta_lake_scan = PyModule::import_bound(py, "daft.delta_lake.delta_lake_scan")?; + let delta_lake_scan_operator = + delta_lake_scan.getattr(pyo3::intern!(py, "DeltaLakeScanOperator"))?; + let delta_lake_operator = delta_lake_scan_operator + .call1((glob_path.as_ref(), py_storage_config))? + .to_object(py); + let scan_operator_handle = + ScanOperatorHandle::from_python_scan_operator(delta_lake_operator, py)?; + Self::table_scan(scan_operator_handle.into(), None) + }) + } + + #[cfg(not(feature = "python"))] + pub fn delta_scan( + glob_path: T, + io_config: Option, + multithreaded_io: bool, + ) -> DaftResult { + panic!("Delta Lake scan requires the 'python' feature to be enabled.") + } + pub fn table_scan( scan_operator: ScanOperatorRef, pushdowns: Option, @@ -142,6 +216,10 @@ impl LogicalPlanBuilder { Ok(Self::new(logical_plan.into(), None)) } + pub fn parquet_scan(glob_path: T) -> ParquetScanBuilder { + ParquetScanBuilder::new(glob_path) + } + pub fn select(&self, to_select: Vec) -> DaftResult { let logical_plan: LogicalPlan = logical_ops::Project::try_new(self.plan.clone(), to_select)?.into(); @@ -498,6 +576,95 @@ impl LogicalPlanBuilder { } } +pub struct ParquetScanBuilder { + pub glob_paths: Vec, + pub infer_schema: bool, + pub coerce_int96_timestamp_unit: TimeUnit, + pub field_id_mapping: Option>>, + pub row_groups: Option>>>, + pub chunk_size: Option, + pub io_config: Option, + pub multithreaded: bool, + pub schema: Option, +} + +impl ParquetScanBuilder { + pub fn new(glob_paths: T) -> Self { + let glob_paths = glob_paths.into_glob_path(); + Self::new_impl(glob_paths) + } + + // concrete implementation to reduce LLVM code duplication + fn new_impl(glob_paths: Vec) -> Self { + Self { + glob_paths, + infer_schema: true, + coerce_int96_timestamp_unit: TimeUnit::Nanoseconds, + field_id_mapping: None, + row_groups: None, + chunk_size: None, + multithreaded: true, + schema: None, + io_config: None, + } + } + pub fn infer_schema(mut self, infer_schema: bool) -> Self { + self.infer_schema = infer_schema; + self + } + pub fn coerce_int96_timestamp_unit(mut self, unit: TimeUnit) -> Self { + self.coerce_int96_timestamp_unit = unit; + self + } + pub fn field_id_mapping(mut self, field_id_mapping: Arc>) -> Self { + self.field_id_mapping = Some(field_id_mapping); + self + } + pub fn row_groups(mut self, row_groups: Vec>>) -> Self { + self.row_groups = Some(row_groups); + self + } + pub fn chunk_size(mut self, chunk_size: usize) -> Self { + self.chunk_size = Some(chunk_size); + self + } + + pub fn io_config(mut self, io_config: IOConfig) -> Self { + self.io_config = Some(io_config); + self + } + + pub fn multithreaded(mut self, multithreaded: bool) -> Self { + self.multithreaded = multithreaded; + self + } + pub fn schema(mut self, schema: SchemaRef) -> Self { + self.schema = Some(schema); + self + } + + pub fn finish(self) -> DaftResult { + let cfg = ParquetSourceConfig { + coerce_int96_timestamp_unit: self.coerce_int96_timestamp_unit, + field_id_mapping: self.field_id_mapping, + row_groups: self.row_groups, + chunk_size: self.chunk_size, + }; + + let operator = Arc::new(GlobScanOperator::try_new( + self.glob_paths, + Arc::new(FileFormatConfig::Parquet(cfg)), + Arc::new(StorageConfig::Native(Arc::new( + NativeStorageConfig::new_internal(self.multithreaded, self.io_config), + ))), + self.infer_schema, + self.schema, + )?); + + LogicalPlanBuilder::table_scan(ScanOperatorRef(operator), None) + } +} + /// A Python-facing wrapper of the LogicalPlanBuilder. /// /// This lightweight proxy interface should hold as much of the Python-specific logic diff --git a/src/daft-plan/src/lib.rs b/src/daft-plan/src/lib.rs index 50e309916b..2541a143db 100644 --- a/src/daft-plan/src/lib.rs +++ b/src/daft-plan/src/lib.rs @@ -19,7 +19,7 @@ pub mod source_info; mod test; mod treenode; -pub use builder::{LogicalPlanBuilder, PyLogicalPlanBuilder}; +pub use builder::{LogicalPlanBuilder, ParquetScanBuilder, PyLogicalPlanBuilder}; pub use daft_core::join::{JoinStrategy, JoinType}; pub use logical_plan::{LogicalPlan, LogicalPlanRef}; pub use partitioning::ClusteringSpec; diff --git a/src/daft-scan/src/lib.rs b/src/daft-scan/src/lib.rs index 10cc0c6804..23191b1d11 100644 --- a/src/daft-scan/src/lib.rs +++ b/src/daft-scan/src/lib.rs @@ -22,7 +22,7 @@ use serde::{Deserialize, Serialize}; mod anonymous; pub use anonymous::AnonymousScanOperator; -mod glob; +pub mod glob; use common_daft_config::DaftExecutionConfig; pub mod scan_task_iters; diff --git a/src/daft-schema/src/time_unit.rs b/src/daft-schema/src/time_unit.rs index d4b17b0e7c..9b1afea2e5 100644 --- a/src/daft-schema/src/time_unit.rs +++ b/src/daft-schema/src/time_unit.rs @@ -1,4 +1,7 @@ +use std::str::FromStr; + use arrow2::datatypes::TimeUnit as ArrowTimeUnit; +use common_error::DaftError; use derive_more::Display; use serde::{Deserialize, Serialize}; @@ -33,6 +36,19 @@ impl TimeUnit { } } +impl FromStr for TimeUnit { + type Err = DaftError; + fn from_str(s: &str) -> Result { + match s.to_lowercase().as_str() { + "ns" | "nanoseconds" => Ok(Self::Nanoseconds), + "us" | "microseconds" => Ok(Self::Microseconds), + "ms" | "milliseconds" => Ok(Self::Milliseconds), + "s" | "seconds" => Ok(Self::Seconds), + _ => Err(DaftError::ValueError("Invalid time unit".to_string())), + } + } +} + impl From<&ArrowTimeUnit> for TimeUnit { fn from(tu: &ArrowTimeUnit) -> Self { match tu { diff --git a/src/daft-sql/Cargo.toml b/src/daft-sql/Cargo.toml index 2b80dda42c..81d7d36ff0 100644 --- a/src/daft-sql/Cargo.toml +++ b/src/daft-sql/Cargo.toml @@ -1,6 +1,7 @@ [dependencies] common-daft-config = {path = "../common/daft-config"} common-error = {path = "../common/error"} +common-io-config = {path = "../common/io-config", default-features = false} daft-core = {path = "../daft-core"} daft-dsl = {path = "../daft-dsl"} daft-functions = {path = "../daft-functions"} diff --git a/src/daft-sql/src/error.rs b/src/daft-sql/src/error.rs index 31f8a400ed..1fd9ae97e7 100644 --- a/src/daft-sql/src/error.rs +++ b/src/daft-sql/src/error.rs @@ -12,6 +12,8 @@ pub enum PlannerError { ParseError { message: String }, #[snafu(display("Invalid operation: {message}"))] InvalidOperation { message: String }, + #[snafu(display("Invalid argument ({message}) for function '{function}'"))] + InvalidFunctionArgument { message: String, function: String }, #[snafu(display("Table not found: {message}"))] TableNotFound { message: String }, #[snafu(display("Column {column_name} not found in {relation}"))] @@ -66,6 +68,13 @@ impl PlannerError { message: message.into(), } } + + pub fn invalid_argument, F: Into>(arg: S, function: F) -> Self { + Self::InvalidFunctionArgument { + message: arg.into(), + function: function.into(), + } + } } #[macro_export] diff --git a/src/daft-sql/src/functions.rs b/src/daft-sql/src/functions.rs index 6b456af17c..2a67d97c63 100644 --- a/src/daft-sql/src/functions.rs +++ b/src/daft-sql/src/functions.rs @@ -1,5 +1,6 @@ use std::{collections::HashMap, sync::Arc}; +use config::SQLModuleConfig; use daft_dsl::ExprRef; use hashing::SQLModuleHashing; use once_cell::sync::Lazy; @@ -31,6 +32,7 @@ pub(crate) static SQL_FUNCTIONS: Lazy = Lazy::new(|| { functions.register::(); functions.register::(); functions.register::(); + functions.register::(); functions }); @@ -110,7 +112,7 @@ pub(crate) struct SQLFunctionArguments { } impl SQLFunctionArguments { - pub fn get_unnamed(&self, idx: usize) -> Option<&ExprRef> { + pub fn get_positional(&self, idx: usize) -> Option<&ExprRef> { self.positional.get(&idx) } pub fn get_named(&self, name: &str) -> Option<&ExprRef> { @@ -123,6 +125,12 @@ impl SQLFunctionArguments { .map(|expr| T::from_expr(expr)) .transpose() } + pub fn try_get_positional(&self, idx: usize) -> Result, PlannerError> { + self.positional + .get(&idx) + .map(|expr| T::from_expr(expr)) + .transpose() + } } pub trait SQLLiteral { @@ -155,6 +163,17 @@ impl SQLLiteral for i64 { } } +impl SQLLiteral for usize { + fn from_expr(expr: &ExprRef) -> Result + where + Self: Sized, + { + expr.as_literal() + .and_then(|lit| lit.as_i64().map(|v| v as Self)) + .ok_or_else(|| PlannerError::invalid_operation("Expected an integer literal")) + } +} + impl SQLLiteral for bool { fn from_expr(expr: &ExprRef) -> Result where @@ -258,6 +277,15 @@ impl SQLPlanner { where T: TryFrom, { + self.parse_function_args(args, expected_named, expected_positional)? + .try_into() + } + pub(crate) fn parse_function_args( + &self, + args: &[FunctionArg], + expected_named: &'static [&'static str], + expected_positional: usize, + ) -> SQLPlannerResult { let mut positional_args = HashMap::new(); let mut named_args = HashMap::new(); for (idx, arg) in args.iter().enumerate() { @@ -282,11 +310,10 @@ impl SQLPlanner { } } - SQLFunctionArguments { + Ok(SQLFunctionArguments { positional: positional_args, named: named_args, - } - .try_into() + }) } pub(crate) fn plan_function_arg( diff --git a/src/daft-sql/src/lib.rs b/src/daft-sql/src/lib.rs index 97fd91c280..6246e8b242 100644 --- a/src/daft-sql/src/lib.rs +++ b/src/daft-sql/src/lib.rs @@ -3,9 +3,9 @@ pub mod error; pub mod functions; mod modules; mod planner; - #[cfg(feature = "python")] pub mod python; +mod table_provider; #[cfg(feature = "python")] use pyo3::prelude::*; diff --git a/src/daft-sql/src/modules/config.rs b/src/daft-sql/src/modules/config.rs new file mode 100644 index 0000000000..9a540d3025 --- /dev/null +++ b/src/daft-sql/src/modules/config.rs @@ -0,0 +1,391 @@ +use common_io_config::{AzureConfig, GCSConfig, HTTPConfig, IOConfig, S3Config}; +use daft_core::prelude::{DataType, Field}; +use daft_dsl::{literal_value, Expr, ExprRef, LiteralValue}; + +use super::SQLModule; +use crate::{ + error::{PlannerError, SQLPlannerResult}, + functions::{SQLFunction, SQLFunctionArguments, SQLFunctions}, + unsupported_sql_err, +}; + +pub struct SQLModuleConfig; + +impl SQLModule for SQLModuleConfig { + fn register(parent: &mut SQLFunctions) { + parent.add_fn("S3Config", S3ConfigFunction); + parent.add_fn("HTTPConfig", HTTPConfigFunction); + parent.add_fn("AzureConfig", AzureConfigFunction); + parent.add_fn("GCSConfig", GCSConfigFunction); + } +} + +pub struct S3ConfigFunction; +macro_rules! item { + ($name:expr, $ty:ident) => { + ( + Field::new(stringify!($name), DataType::$ty), + literal_value($name), + ) + }; +} + +impl SQLFunction for S3ConfigFunction { + fn to_expr( + &self, + inputs: &[sqlparser::ast::FunctionArg], + planner: &crate::planner::SQLPlanner, + ) -> crate::error::SQLPlannerResult { + // TODO(cory): Ideally we should use serde to deserialize the input arguments + let args: SQLFunctionArguments = planner.parse_function_args( + inputs, + &[ + "region_name", + "endpoint_url", + "key_id", + "session_token", + "access_key", + "credentials_provider", + "buffer_time", + "max_connections_per_io_thread", + "retry_initial_backoff_ms", + "connect_timeout_ms", + "read_timeout_ms", + "num_tries", + "retry_mode", + "anonymous", + "use_ssl", + "verify_ssl", + "check_hostname_ssl", + "requester_pays", + "force_virtual_addressing", + "profile_name", + ], + 0, + )?; + + let region_name = args.try_get_named::("region_name")?; + let endpoint_url = args.try_get_named::("endpoint_url")?; + let key_id = args.try_get_named::("key_id")?; + let session_token = args.try_get_named::("session_token")?; + + let access_key = args.try_get_named::("access_key")?; + let buffer_time = args.try_get_named("buffer_time")?.map(|t: i64| t as u64); + + let max_connections_per_io_thread = args + .try_get_named("max_connections_per_io_thread")? + .map(|t: i64| t as u32); + + let retry_initial_backoff_ms = args + .try_get_named("retry_initial_backoff_ms")? + .map(|t: i64| t as u64); + + let connect_timeout_ms = args + .try_get_named("connect_timeout_ms")? + .map(|t: i64| t as u64); + + let read_timeout_ms = args + .try_get_named("read_timeout_ms")? + .map(|t: i64| t as u64); + + let num_tries = args.try_get_named("num_tries")?.map(|t: i64| t as u32); + let retry_mode = args.try_get_named::("retry_mode")?; + let anonymous = args.try_get_named::("anonymous")?; + let use_ssl = args.try_get_named::("use_ssl")?; + let verify_ssl = args.try_get_named::("verify_ssl")?; + let check_hostname_ssl = args.try_get_named::("check_hostname_ssl")?; + let requester_pays = args.try_get_named::("requester_pays")?; + let force_virtual_addressing = args.try_get_named::("force_virtual_addressing")?; + let profile_name = args.try_get_named::("profile_name")?; + + let entries = vec![ + (Field::new("variant", DataType::Utf8), literal_value("s3")), + item!(region_name, Utf8), + item!(endpoint_url, Utf8), + item!(key_id, Utf8), + item!(session_token, Utf8), + item!(access_key, Utf8), + item!(buffer_time, UInt64), + item!(max_connections_per_io_thread, UInt32), + item!(retry_initial_backoff_ms, UInt64), + item!(connect_timeout_ms, UInt64), + item!(read_timeout_ms, UInt64), + item!(num_tries, UInt32), + item!(retry_mode, Utf8), + item!(anonymous, Boolean), + item!(use_ssl, Boolean), + item!(verify_ssl, Boolean), + item!(check_hostname_ssl, Boolean), + item!(requester_pays, Boolean), + item!(force_virtual_addressing, Boolean), + item!(profile_name, Utf8), + ] + .into_iter() + .collect::<_>(); + + Ok(Expr::Literal(LiteralValue::Struct(entries)).arced()) + } +} + +pub struct HTTPConfigFunction; + +impl SQLFunction for HTTPConfigFunction { + fn to_expr( + &self, + inputs: &[sqlparser::ast::FunctionArg], + planner: &crate::planner::SQLPlanner, + ) -> crate::error::SQLPlannerResult { + let args: SQLFunctionArguments = + planner.parse_function_args(inputs, &["user_agent", "bearer_token"], 0)?; + + let user_agent = args.try_get_named::("user_agent")?; + let bearer_token = args.try_get_named::("bearer_token")?; + + let entries = vec![ + (Field::new("variant", DataType::Utf8), literal_value("http")), + item!(user_agent, Utf8), + item!(bearer_token, Utf8), + ] + .into_iter() + .collect::<_>(); + + Ok(Expr::Literal(LiteralValue::Struct(entries)).arced()) + } +} +pub struct AzureConfigFunction; +impl SQLFunction for AzureConfigFunction { + fn to_expr( + &self, + inputs: &[sqlparser::ast::FunctionArg], + planner: &crate::planner::SQLPlanner, + ) -> crate::error::SQLPlannerResult { + let args: SQLFunctionArguments = planner.parse_function_args( + inputs, + &[ + "storage_account", + "access_key", + "sas_token", + "bearer_token", + "tenant_id", + "client_id", + "client_secret", + "use_fabric_endpoint", + "anonymous", + "endpoint_url", + "use_ssl", + ], + 0, + )?; + + let storage_account = args.try_get_named::("storage_account")?; + let access_key = args.try_get_named::("access_key")?; + let sas_token = args.try_get_named::("sas_token")?; + let bearer_token = args.try_get_named::("bearer_token")?; + let tenant_id = args.try_get_named::("tenant_id")?; + let client_id = args.try_get_named::("client_id")?; + let client_secret = args.try_get_named::("client_secret")?; + let use_fabric_endpoint = args.try_get_named::("use_fabric_endpoint")?; + let anonymous = args.try_get_named::("anonymous")?; + let endpoint_url = args.try_get_named::("endpoint_url")?; + let use_ssl = args.try_get_named::("use_ssl")?; + + let entries = vec![ + ( + Field::new("variant", DataType::Utf8), + literal_value("azure"), + ), + item!(storage_account, Utf8), + item!(access_key, Utf8), + item!(sas_token, Utf8), + item!(bearer_token, Utf8), + item!(tenant_id, Utf8), + item!(client_id, Utf8), + item!(client_secret, Utf8), + item!(use_fabric_endpoint, Boolean), + item!(anonymous, Boolean), + item!(endpoint_url, Utf8), + item!(use_ssl, Boolean), + ] + .into_iter() + .collect::<_>(); + + Ok(Expr::Literal(LiteralValue::Struct(entries)).arced()) + } +} + +pub struct GCSConfigFunction; + +impl SQLFunction for GCSConfigFunction { + fn to_expr( + &self, + inputs: &[sqlparser::ast::FunctionArg], + planner: &crate::planner::SQLPlanner, + ) -> SQLPlannerResult { + let args: SQLFunctionArguments = planner.parse_function_args( + inputs, + &["project_id", "credentials", "token", "anonymous"], + 0, + )?; + + let project_id = args.try_get_named::("project_id")?; + let credentials = args.try_get_named::("credentials")?; + let token = args.try_get_named::("token")?; + let anonymous = args.try_get_named::("anonymous")?; + + let entries = vec![ + (Field::new("variant", DataType::Utf8), literal_value("gcs")), + item!(project_id, Utf8), + item!(credentials, Utf8), + item!(token, Utf8), + item!(anonymous, Boolean), + ] + .into_iter() + .collect::<_>(); + + Ok(Expr::Literal(LiteralValue::Struct(entries)).arced()) + } +} + +pub(crate) fn expr_to_iocfg(expr: &ExprRef) -> SQLPlannerResult { + // TODO(CORY): use serde to deserialize this + let Expr::Literal(LiteralValue::Struct(entries)) = expr.as_ref() else { + unsupported_sql_err!("Invalid IOConfig"); + }; + + macro_rules! get_value { + ($field:literal, $type:ident) => { + entries + .get(&Field::new($field, DataType::$type)) + .and_then(|s| match s { + LiteralValue::$type(s) => Some(Ok(s.clone())), + LiteralValue::Null => None, + _ => Some(Err(PlannerError::invalid_argument($field, "IOConfig"))), + }) + .transpose() + }; + } + + let variant = get_value!("variant", Utf8)? + .expect("variant is required for IOConfig, this indicates a programming error"); + + match variant.as_ref() { + "s3" => { + let region_name = get_value!("region_name", Utf8)?; + let endpoint_url = get_value!("endpoint_url", Utf8)?; + let key_id = get_value!("key_id", Utf8)?; + let session_token = get_value!("session_token", Utf8)?.map(|s| s.into()); + let access_key = get_value!("access_key", Utf8)?.map(|s| s.into()); + let buffer_time = get_value!("buffer_time", UInt64)?; + let max_connections_per_io_thread = + get_value!("max_connections_per_io_thread", UInt32)?; + let retry_initial_backoff_ms = get_value!("retry_initial_backoff_ms", UInt64)?; + let connect_timeout_ms = get_value!("connect_timeout_ms", UInt64)?; + let read_timeout_ms = get_value!("read_timeout_ms", UInt64)?; + let num_tries = get_value!("num_tries", UInt32)?; + let retry_mode = get_value!("retry_mode", Utf8)?; + let anonymous = get_value!("anonymous", Boolean)?; + let use_ssl = get_value!("use_ssl", Boolean)?; + let verify_ssl = get_value!("verify_ssl", Boolean)?; + let check_hostname_ssl = get_value!("check_hostname_ssl", Boolean)?; + let requester_pays = get_value!("requester_pays", Boolean)?; + let force_virtual_addressing = get_value!("force_virtual_addressing", Boolean)?; + let profile_name = get_value!("profile_name", Utf8)?; + let default = S3Config::default(); + let s3_config = S3Config { + region_name, + endpoint_url, + key_id, + session_token, + access_key, + credentials_provider: None, + buffer_time, + max_connections_per_io_thread: max_connections_per_io_thread + .unwrap_or(default.max_connections_per_io_thread), + retry_initial_backoff_ms: retry_initial_backoff_ms + .unwrap_or(default.retry_initial_backoff_ms), + connect_timeout_ms: connect_timeout_ms.unwrap_or(default.connect_timeout_ms), + read_timeout_ms: read_timeout_ms.unwrap_or(default.read_timeout_ms), + num_tries: num_tries.unwrap_or(default.num_tries), + retry_mode, + anonymous: anonymous.unwrap_or(default.anonymous), + use_ssl: use_ssl.unwrap_or(default.use_ssl), + verify_ssl: verify_ssl.unwrap_or(default.verify_ssl), + check_hostname_ssl: check_hostname_ssl.unwrap_or(default.check_hostname_ssl), + requester_pays: requester_pays.unwrap_or(default.requester_pays), + force_virtual_addressing: force_virtual_addressing + .unwrap_or(default.force_virtual_addressing), + profile_name, + }; + + Ok(IOConfig { + s3: s3_config, + ..Default::default() + }) + } + "http" => { + let default = HTTPConfig::default(); + let user_agent = get_value!("user_agent", Utf8)?.unwrap_or(default.user_agent); + let bearer_token = get_value!("bearer_token", Utf8)?.map(|s| s.into()); + + Ok(IOConfig { + http: HTTPConfig { + user_agent, + bearer_token, + }, + ..Default::default() + }) + } + "azure" => { + let storage_account = get_value!("storage_account", Utf8)?; + let access_key = get_value!("access_key", Utf8)?; + let sas_token = get_value!("sas_token", Utf8)?; + let bearer_token = get_value!("bearer_token", Utf8)?; + let tenant_id = get_value!("tenant_id", Utf8)?; + let client_id = get_value!("client_id", Utf8)?; + let client_secret = get_value!("client_secret", Utf8)?; + let use_fabric_endpoint = get_value!("use_fabric_endpoint", Boolean)?; + let anonymous = get_value!("anonymous", Boolean)?; + let endpoint_url = get_value!("endpoint_url", Utf8)?; + let use_ssl = get_value!("use_ssl", Boolean)?; + + let default = AzureConfig::default(); + + Ok(IOConfig { + azure: AzureConfig { + storage_account, + access_key: access_key.map(|s| s.into()), + sas_token, + bearer_token, + tenant_id, + client_id, + client_secret: client_secret.map(|s| s.into()), + use_fabric_endpoint: use_fabric_endpoint.unwrap_or(default.use_fabric_endpoint), + anonymous: anonymous.unwrap_or(default.anonymous), + endpoint_url, + use_ssl: use_ssl.unwrap_or(default.use_ssl), + }, + ..Default::default() + }) + } + "gcs" => { + let project_id = get_value!("project_id", Utf8)?; + let credentials = get_value!("credentials", Utf8)?; + let token = get_value!("token", Utf8)?; + let anonymous = get_value!("anonymous", Boolean)?; + let default = GCSConfig::default(); + + Ok(IOConfig { + gcs: GCSConfig { + project_id, + credentials: credentials.map(|s| s.into()), + token, + anonymous: anonymous.unwrap_or(default.anonymous), + }, + ..Default::default() + }) + } + _ => { + unreachable!("variant is required for IOConfig, this indicates a programming error") + } + } +} diff --git a/src/daft-sql/src/modules/image/resize.rs b/src/daft-sql/src/modules/image/resize.rs index e4c9804d39..ac6c12fd50 100644 --- a/src/daft-sql/src/modules/image/resize.rs +++ b/src/daft-sql/src/modules/image/resize.rs @@ -16,7 +16,7 @@ impl TryFrom for ImageResize { fn try_from(args: SQLFunctionArguments) -> Result { let width = args .get_named("w") - .or_else(|| args.get_unnamed(0)) + .or_else(|| args.get_positional(0)) .map(|arg| match arg.as_ref() { Expr::Literal(LiteralValue::Int64(i)) => Ok(*i), _ => unsupported_sql_err!("Expected width to be a number"), @@ -28,7 +28,7 @@ impl TryFrom for ImageResize { let height = args .get_named("h") - .or_else(|| args.get_unnamed(1)) + .or_else(|| args.get_positional(1)) .map(|arg| match arg.as_ref() { Expr::Literal(LiteralValue::Int64(i)) => Ok(*i), _ => unsupported_sql_err!("Expected height to be a number"), diff --git a/src/daft-sql/src/modules/mod.rs b/src/daft-sql/src/modules/mod.rs index 989c401393..af4cb731a3 100644 --- a/src/daft-sql/src/modules/mod.rs +++ b/src/daft-sql/src/modules/mod.rs @@ -1,6 +1,7 @@ use crate::functions::SQLFunctions; pub mod aggs; +pub mod config; pub mod float; pub mod hashing; pub mod image; diff --git a/src/daft-sql/src/planner.rs b/src/daft-sql/src/planner.rs index afeb14fa2d..1be5b724a9 100644 --- a/src/daft-sql/src/planner.rs +++ b/src/daft-sql/src/planner.rs @@ -397,7 +397,19 @@ impl SQLPlanner { fn plan_relation(&self, rel: &sqlparser::ast::TableFactor) -> SQLPlannerResult { match rel { - sqlparser::ast::TableFactor::Table { name, .. } => { + sqlparser::ast::TableFactor::Table { + name, + args: Some(args), + alias, + .. + } => { + let tbl_fn = name.0.first().unwrap().value.as_str(); + + self.plan_table_function(tbl_fn, args, alias) + } + sqlparser::ast::TableFactor::Table { + name, args: None, .. + } => { let table_name = name.to_string(); let plan = self .catalog @@ -728,7 +740,22 @@ impl SQLPlanner { } SQLExpr::Struct { .. } => unsupported_sql_err!("STRUCT"), SQLExpr::Named { .. } => unsupported_sql_err!("NAMED"), - SQLExpr::Dictionary(_) => unsupported_sql_err!("DICTIONARY"), + SQLExpr::Dictionary(dict) => { + let entries = dict + .iter() + .map(|entry| { + let key = entry.key.value.clone(); + let value = self.plan_expr(&entry.value)?; + let value = value.as_literal().ok_or_else(|| { + PlannerError::invalid_operation("Dictionary value is not a literal") + })?; + let struct_field = Field::new(key, value.get_type()); + Ok((struct_field, value.clone())) + }) + .collect::>()?; + + Ok(Expr::Literal(LiteralValue::Struct(entries)).arced()) + } SQLExpr::Map(_) => unsupported_sql_err!("MAP"), SQLExpr::Subscript { expr, subscript } => self.plan_subscript(expr, subscript.as_ref()), SQLExpr::Array(_) => unsupported_sql_err!("ARRAY"), diff --git a/src/daft-sql/src/table_provider/mod.rs b/src/daft-sql/src/table_provider/mod.rs new file mode 100644 index 0000000000..453fa4965f --- /dev/null +++ b/src/daft-sql/src/table_provider/mod.rs @@ -0,0 +1,119 @@ +pub mod read_parquet; +use std::{collections::HashMap, sync::Arc}; + +use daft_plan::LogicalPlanBuilder; +use once_cell::sync::Lazy; +use read_parquet::ReadParquetFunction; +use sqlparser::ast::{TableAlias, TableFunctionArgs}; + +use crate::{ + error::SQLPlannerResult, + modules::config::expr_to_iocfg, + planner::{Relation, SQLPlanner}, + unsupported_sql_err, +}; + +pub(crate) static SQL_TABLE_FUNCTIONS: Lazy = Lazy::new(|| { + let mut functions = SQLTableFunctions::new(); + functions.add_fn("read_parquet", ReadParquetFunction); + #[cfg(feature = "python")] + functions.add_fn("read_deltalake", ReadDeltalakeFunction); + + functions +}); + +/// TODOs +/// - Use multimap for function variants. +/// - Add more functions.. +pub struct SQLTableFunctions { + pub(crate) map: HashMap>, +} + +impl SQLTableFunctions { + /// Create a new [SQLFunctions] instance. + pub fn new() -> Self { + Self { + map: HashMap::new(), + } + } + /// Add a [FunctionExpr] to the [SQLFunctions] instance. + pub(crate) fn add_fn(&mut self, name: &str, func: F) { + self.map.insert(name.to_string(), Arc::new(func)); + } + + /// Get a function by name from the [SQLFunctions] instance. + pub(crate) fn get(&self, name: &str) -> Option<&Arc> { + self.map.get(name) + } +} + +impl SQLPlanner { + pub(crate) fn plan_table_function( + &self, + fn_name: &str, + args: &TableFunctionArgs, + alias: &Option, + ) -> SQLPlannerResult { + let fns = &SQL_TABLE_FUNCTIONS; + + let Some(func) = fns.get(fn_name) else { + unsupported_sql_err!("Function `{}` not found", fn_name); + }; + + let builder = func.plan(self, args)?; + let name = alias + .as_ref() + .map(|a| a.name.value.clone()) + .unwrap_or_else(|| fn_name.to_string()); + + Ok(Relation::new(builder, name)) + } +} + +pub(crate) trait SQLTableFunction: Send + Sync { + fn plan( + &self, + planner: &SQLPlanner, + args: &TableFunctionArgs, + ) -> SQLPlannerResult; +} + +pub struct ReadDeltalakeFunction; + +#[cfg(feature = "python")] +impl SQLTableFunction for ReadDeltalakeFunction { + fn plan( + &self, + planner: &SQLPlanner, + args: &TableFunctionArgs, + ) -> SQLPlannerResult { + let (uri, io_config) = match args.args.as_slice() { + [uri] => (uri, None), + [uri, io_config] => { + let args = planner.parse_function_args(&[io_config.clone()], &["io_config"], 0)?; + let io_config = args.get_named("io_config").map(expr_to_iocfg).transpose()?; + + (uri, io_config) + } + _ => unsupported_sql_err!("Expected one or two arguments"), + }; + let uri = planner.plan_function_arg(uri)?; + + let Some(uri) = uri.as_literal().and_then(|lit| lit.as_str()) else { + unsupported_sql_err!("Expected a string literal for the first argument"); + }; + + LogicalPlanBuilder::delta_scan(uri, io_config, true).map_err(From::from) + } +} + +#[cfg(not(feature = "python"))] +impl SQLTableFunction for ReadDeltalakeFunction { + fn plan( + &self, + planner: &SQLPlanner, + args: &TableFunctionArgs, + ) -> SQLPlannerResult { + unsupported_sql_err!("`read_deltalake` function is not supported. Enable the `python` feature to use this function.") + } +} diff --git a/src/daft-sql/src/table_provider/read_parquet.rs b/src/daft-sql/src/table_provider/read_parquet.rs new file mode 100644 index 0000000000..36f84507a0 --- /dev/null +++ b/src/daft-sql/src/table_provider/read_parquet.rs @@ -0,0 +1,77 @@ +use daft_core::prelude::TimeUnit; +use daft_plan::{LogicalPlanBuilder, ParquetScanBuilder}; +use sqlparser::ast::TableFunctionArgs; + +use super::SQLTableFunction; +use crate::{ + error::{PlannerError, SQLPlannerResult}, + functions::SQLFunctionArguments, + modules::config::expr_to_iocfg, + planner::SQLPlanner, +}; + +pub(super) struct ReadParquetFunction; + +impl TryFrom for ParquetScanBuilder { + type Error = PlannerError; + + fn try_from(args: SQLFunctionArguments) -> Result { + let glob_paths: String = args.try_get_positional(0)?.ok_or_else(|| { + PlannerError::invalid_operation("path is required for `read_parquet`") + })?; + let infer_schema = args.try_get_named("infer_schema")?.unwrap_or(true); + let coerce_int96_timestamp_unit = + args.try_get_named::("coerce_int96_timestamp_unit")?; + let coerce_int96_timestamp_unit: TimeUnit = coerce_int96_timestamp_unit + .as_deref() + .unwrap_or("nanoseconds") + .parse::() + .map_err(|_| { + PlannerError::invalid_argument("coerce_int96_timestamp_unit", "read_parquet") + })?; + let chunk_size = args.try_get_named("chunk_size")?; + let multithreaded = args.try_get_named("multithreaded")?.unwrap_or(true); + + let field_id_mapping = None; // TODO + let row_groups = None; // TODO + let schema = None; // TODO + let io_config = args.get_named("io_config").map(expr_to_iocfg).transpose()?; + + Ok(Self { + glob_paths: vec![glob_paths], + infer_schema, + coerce_int96_timestamp_unit, + field_id_mapping, + row_groups, + chunk_size, + io_config, + multithreaded, + schema, + }) + } +} + +impl SQLTableFunction for ReadParquetFunction { + fn plan( + &self, + planner: &SQLPlanner, + args: &TableFunctionArgs, + ) -> SQLPlannerResult { + let builder: ParquetScanBuilder = planner.plan_function_args( + args.args.as_slice(), + &[ + "infer_schema", + "coerce_int96_timestamp_unit", + "chunk_size", + "multithreaded", + // "schema", + // "field_id_mapping", + // "row_groups", + "io_config", + ], + 1, // 1 positional argument (path) + )?; + + builder.finish().map_err(From::from) + } +} diff --git a/tests/sql/test_table_funcs.py b/tests/sql/test_table_funcs.py new file mode 100644 index 0000000000..16fbf040c7 --- /dev/null +++ b/tests/sql/test_table_funcs.py @@ -0,0 +1,7 @@ +import daft + + +def test_sql_read_parquet(): + df = daft.sql("SELECT * FROM read_parquet('tests/assets/parquet-data/mvp.parquet')").collect() + expected = daft.read_parquet("tests/assets/parquet-data/mvp.parquet").collect() + assert df.to_pydict() == expected.to_pydict() From f3b998e8a5726251397888c986c45b27bdd0cad7 Mon Sep 17 00:00:00 2001 From: Cory Grinstead Date: Thu, 3 Oct 2024 12:53:46 -0500 Subject: [PATCH 4/5] [BUG]: error messages for add (#2990) if you tried to add utf8 to another unsupported dtype, it would result in the incorrect error message of 'Cannot add types: {other}, {other}' instead of 'Cannot add types: Utf8, {other}' this fixes that error message --- src/daft-core/src/datatypes/infer_datatype.rs | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/daft-core/src/datatypes/infer_datatype.rs b/src/daft-core/src/datatypes/infer_datatype.rs index 9c05eb0b02..ab80f4eac4 100644 --- a/src/daft-core/src/datatypes/infer_datatype.rs +++ b/src/daft-core/src/datatypes/infer_datatype.rs @@ -138,27 +138,27 @@ impl<'a> Add for InferDataType<'a> { (du_self @ &DataType::Duration(..), du_other @ &DataType::Duration(..)) => Err(DaftError::TypeError( format!("Cannot add due to differing precision: {}, {}. Please explicitly cast to the precision you wish to add in.", du_self, du_other) )), - (DataType::Null, other) | (other, DataType::Null) => { + (dtype @ DataType::Null, other) | (other, dtype @ DataType::Null) => { match other { // Condition is for backwards compatibility. TODO: remove DataType::Binary | DataType::FixedSizeBinary(..) | DataType::Date => Err(DaftError::TypeError( - format!("Cannot add types: {}, {}", self, other) + format!("Cannot add types: {}, {}", dtype, other) )), other if other.is_physical() => Ok(other.clone()), _ => Err(DaftError::TypeError( - format!("Cannot add types: {}, {}", self, other) + format!("Cannot add types: {}, {}", dtype, other) )), } } - (DataType::Utf8, other) | (other, DataType::Utf8) => { + (dtype @ DataType::Utf8, other) | (other, dtype @ DataType::Utf8) => { match other { // DataType::Date condition is for backwards compatibility. TODO: remove DataType::Binary | DataType::FixedSizeBinary(..) | DataType::Date => Err(DaftError::TypeError( - format!("Cannot add types: {}, {}", self, other) + format!("Cannot add types: {}, {}", dtype, other) )), other if other.is_physical() => Ok(DataType::Utf8), _ => Err(DaftError::TypeError( - format!("Cannot add types: {}, {}", self, other) + format!("Cannot add types: {}, {}", dtype, other) )), } } From 62d05814f38c7b848b8e345f7e70128034b6d00d Mon Sep 17 00:00:00 2001 From: Anmol Singh Date: Thu, 3 Oct 2024 11:17:04 -0700 Subject: [PATCH 5/5] [BUG] Fix join errors with same key name joins (resolves #2649) (#2877) The issue fixed here had a workaround previously - aliasing the duplicate column name. This is not needed anymore as the aliasing is performed under the hood, taking care of uniqueness of individual column keys to avoid the duplicate issue. --------- Co-authored-by: AnmolS --- Cargo.lock | 1 + src/daft-plan/Cargo.toml | 1 + src/daft-plan/src/logical_ops/join.rs | 86 ++++++++++++++++++++++++--- tests/dataframe/test_joins.py | 11 ++++ 4 files changed, 92 insertions(+), 7 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 8d60981bd3..db446053c8 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2084,6 +2084,7 @@ dependencies = [ "serde", "snafu", "test-log", + "uuid 1.10.0", ] [[package]] diff --git a/src/daft-plan/Cargo.toml b/src/daft-plan/Cargo.toml index 7de191fe81..13d2c4307f 100644 --- a/src/daft-plan/Cargo.toml +++ b/src/daft-plan/Cargo.toml @@ -34,6 +34,7 @@ log = {workspace = true} pyo3 = {workspace = true, optional = true} serde = {workspace = true, features = ["rc"]} snafu = {workspace = true} +uuid = {version = "1", features = ["v4"]} [dev-dependencies] daft-dsl = {path = "../daft-dsl", features = ["test-utils"]} diff --git a/src/daft-plan/src/logical_ops/join.rs b/src/daft-plan/src/logical_ops/join.rs index 2a68390066..8e6d0b005e 100644 --- a/src/daft-plan/src/logical_ops/join.rs +++ b/src/daft-plan/src/logical_ops/join.rs @@ -3,7 +3,7 @@ use std::{ sync::Arc, }; -use common_error::DaftError; +use common_error::{DaftError, DaftResult}; use daft_core::prelude::*; use daft_dsl::{ col, @@ -13,6 +13,7 @@ use daft_dsl::{ }; use itertools::Itertools; use snafu::ResultExt; +use uuid::Uuid; use crate::{ logical_ops::Project, @@ -54,14 +55,31 @@ impl Join { join_type: JoinType, join_strategy: Option, ) -> logical_plan::Result { - let (left_on, left_fields) = - resolve_exprs(left_on, &left.schema(), false).context(CreationSnafu)?; - let (right_on, right_fields) = + let (left_on, _) = resolve_exprs(left_on, &left.schema(), false).context(CreationSnafu)?; + let (right_on, _) = resolve_exprs(right_on, &right.schema(), false).context(CreationSnafu)?; - for (on_exprs, on_fields) in [(&left_on, left_fields), (&right_on, right_fields)] { - let on_schema = Schema::new(on_fields).context(CreationSnafu)?; - for (field, expr) in on_schema.fields.values().zip(on_exprs.iter()) { + let (unique_left_on, unique_right_on) = + Self::rename_join_keys(left_on.clone(), right_on.clone()); + + let left_fields: Vec = unique_left_on + .iter() + .map(|e| e.to_field(&left.schema())) + .collect::>>() + .context(CreationSnafu)?; + + let right_fields: Vec = unique_right_on + .iter() + .map(|e| e.to_field(&right.schema())) + .collect::>>() + .context(CreationSnafu)?; + + for (on_exprs, on_fields) in [ + (&unique_left_on, &left_fields), + (&unique_right_on, &right_fields), + ] { + for (field, expr) in on_fields.iter().zip(on_exprs.iter()) { + // Null type check for both fields and expressions if matches!(field.dtype, DataType::Null) { return Err(DaftError::ValueError(format!( "Can't join on null type expressions: {expr}" @@ -167,6 +185,60 @@ impl Join { } } + /// Renames join keys for the given left and right expressions. This is required to + /// prevent errors when the join keys on the left and right expressions have the same key + /// name. + /// + /// This function takes two vectors of expressions (`left_exprs` and `right_exprs`) and + /// checks for pairs of column expressions that differ. If both expressions in a pair + /// are column expressions and they are not identical, it generates a unique identifier + /// and renames both expressions by appending this identifier to their original names. + /// + /// The function returns two vectors of expressions, where the renamed expressions are + /// substituted for the original expressions in the cases where renaming occurred. + /// + /// # Parameters + /// - `left_exprs`: A vector of expressions from the left side of a join. + /// - `right_exprs`: A vector of expressions from the right side of a join. + /// + /// # Returns + /// A tuple containing two vectors of expressions, one for the left side and one for the + /// right side, where expressions that needed to be renamed have been modified. + /// + /// # Example + /// ``` + /// let (renamed_left, renamed_right) = rename_join_keys(left_expressions, right_expressions); + /// ``` + /// + /// For more details, see [issue #2649](https://github.com/Eventual-Inc/Daft/issues/2649). + + fn rename_join_keys( + left_exprs: Vec>, + right_exprs: Vec>, + ) -> (Vec>, Vec>) { + left_exprs + .into_iter() + .zip(right_exprs) + .map( + |(left_expr, right_expr)| match (&*left_expr, &*right_expr) { + (Expr::Column(left_name), Expr::Column(right_name)) + if left_name == right_name => + { + (left_expr, right_expr) + } + _ => { + let unique_id = Uuid::new_v4().to_string(); + let renamed_left_expr = + left_expr.alias(format!("{}_{}", left_expr.name(), unique_id)); + let renamed_right_expr = + right_expr.alias(format!("{}_{}", right_expr.name(), unique_id)); + (renamed_left_expr, renamed_right_expr) + } + }, + ) + .unzip() + } + pub fn multiline_display(&self) -> Vec { let mut res = vec![]; res.push(format!("Join: Type = {}", self.join_type)); diff --git a/tests/dataframe/test_joins.py b/tests/dataframe/test_joins.py index b0bdbf9df4..5e79acf698 100644 --- a/tests/dataframe/test_joins.py +++ b/tests/dataframe/test_joins.py @@ -53,6 +53,17 @@ def test_columns_after_join(make_df): assert set(joined_df2.schema().column_names()) == set(["A", "B"]) +def test_rename_join_keys_in_dataframe(make_df): + df1 = make_df({"A": [1, 2], "B": [2, 2]}) + + df2 = make_df({"A": [1, 2]}) + joined_df1 = df1.join(df2, left_on=["A", "B"], right_on=["A", "A"]) + joined_df2 = df1.join(df2, left_on=["B", "A"], right_on=["A", "A"]) + + assert set(joined_df1.schema().column_names()) == set(["A", "B"]) + assert set(joined_df2.schema().column_names()) == set(["A", "B"]) + + @pytest.mark.parametrize("n_partitions", [1, 2, 4]) @pytest.mark.parametrize( "join_strategy",