diff --git a/.gitignore b/.gitignore index 1e47926b2..506931c98 100644 --- a/.gitignore +++ b/.gitignore @@ -58,12 +58,14 @@ docs/example1.dat docs/example3.dat python/.eggs/ python/doc/ +python/examples/.ipynb_checkpoints # Egg metadata *.egg-info .vscode .idea/ .pytest_cache/ +.ruff_cache/ pkgs docker_cache .gdb_history diff --git a/python/README.md b/python/README.md index 0a26ee979..8ba7d1f27 100644 --- a/python/README.md +++ b/python/README.md @@ -79,6 +79,102 @@ pyarrow_batches = df.collect() Check [DataFusion python](https://datafusion.apache.org/python/) provides more examples and manuals. +## Jupyter Notebook Support + +PyBallista provides first-class Jupyter notebook support with SQL magic commands and rich HTML rendering. + +### Install Jupyter extras first: +```bash +pip install "ballista[jupyter]" +``` + +### HTML Table Rendering + +DataFrames automatically render as styled HTML tables in Jupyter notebooks: + +```python +from ballista import BallistaSessionContext + +ctx = BallistaSessionContext("df://localhost:50050") +df = ctx.sql("SELECT * FROM my_table LIMIT 10") +df # Renders as HTML table via _repr_html_() +``` + +### SQL Magic Commands + +For a more interactive SQL experience, load the Ballista Jupyter extension: + +```python +# Load the extension +%load_ext ballista.jupyter + +# Connect to a Ballista cluster +%ballista connect df://localhost:50050 + +# Register .parquet table +%register parquet public.test_data_v1 ../testdata/test.parquet + +# Check connection status +%ballista status + +# List registered tables +%ballista tables + +# Show table schema +%ballista schema my_table + +# Execute a simple query (line magic) +%sql SELECT COUNT(*) FROM orders + +# Execute a complex query (cell magic) +%%sql +SELECT + customer_id, + SUM(amount) as total +FROM orders +GROUP BY customer_id +ORDER BY total DESC +LIMIT 10 +``` + +You can also store results in a variable: + +```python +%%sql my_result +SELECT * FROM orders WHERE status = 'pending' +``` + +### Execution Plan Visualization + +Visualize query execution plans directly in notebooks: + +```python +df = ctx.sql("SELECT * FROM orders WHERE amount > 100") +df.explain_visual() # Displays SVG visualization + +# With runtime statistics +df.explain_visual(analyze=True) +``` + +> **Note:** Full SVG visualization requires graphviz to be installed (`brew install graphviz` on macOS). + +### Progress Indicators + +For long-running queries, use `collect_with_progress()` to see execution status: + +```python +df = ctx.sql("SELECT * FROM large_table") +batches = df.collect_with_progress() +``` + +### Example Notebooks + +See the `examples/` directory for Jupyter notebooks demonstrating various features: + +- `getting_started.ipynb` - Basic connection and queries +- `dataframe_api.ipynb` - DataFrame transformations +- `distributed_queries.ipynb` - Multi-stage distributed query examples + ## Scheduler and Executor Scheduler and executors can be configured and started from python code. diff --git a/python/examples/dataframe_api.ipynb b/python/examples/dataframe_api.ipynb new file mode 100644 index 000000000..aaa86df75 --- /dev/null +++ b/python/examples/dataframe_api.ipynb @@ -0,0 +1,419 @@ +{ + "cells": [ + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "\n", + "\n", + "# DataFrame API with Ballista\n", + "\n", + "This notebook demonstrates the DataFrame API available in Ballista.\n", + "\n", + "The DataFrame API provides a programmatic way to build queries, which can be\n", + "more convenient than writing SQL for complex transformations." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "from ballista import BallistaSessionContext, setup_test_cluster\n", + "from datafusion import col, lit\n", + "from datafusion import functions as f\n", + "\n", + "# Set up test cluster and connect\n", + "host, port = setup_test_cluster()\n", + "ctx = BallistaSessionContext(f\"df://{host}:{port}\")\n", + "\n", + "# Register sample data\n", + "ctx.register_parquet(\"test_data\", \"../testdata/test.parquet\")\n", + "ctx.register_csv(\"csv_data\", \"../testdata/test.csv\", has_header=True)\n", + "\n", + "print(f\"Connected! Session ID: {ctx.session_id}\")" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Basic Operations" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Read a table as a DataFrame\n", + "df = ctx.table(\"test_data\")\n", + "\n", + "# Display schema\n", + "print(\"Schema:\")\n", + "for field in df.schema():\n", + " print(f\" {field.name}: {field.type}\")" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Show first few rows\n", + "df.show(5)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Selecting Columns" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Select specific columns by name\n", + "df.select(\"id\", \"bool_col\", \"tinyint_col\")" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Select with column expressions\n", + "df.select(\n", + " col(\"id\"),\n", + " col(\"tinyint_col\").alias(\"tiny\"),\n", + " (col(\"id\") * lit(10)).alias(\"id_times_10\")\n", + ")" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Filtering Data" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Simple filter\n", + "df.filter(col(\"id\") > lit(4))" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Complex filter with AND/OR\n", + "df.filter(\n", + " (col(\"id\") >= lit(2)) & (col(\"id\") <= lit(5))\n", + ")" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Filter with boolean column\n", + "df.filter(col(\"bool_col\") == lit(True))" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Sorting" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Sort ascending\n", + "df.sort(col(\"id\").sort(ascending=True))" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Sort descending\n", + "df.sort(col(\"id\").sort(ascending=False))" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Limiting Results" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Limit number of rows\n", + "df.limit(3)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Aggregations" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Count all rows\n", + "result = df.aggregate([], [f.count_star().alias(\"total_count\")])\n", + "result" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Group by and aggregate\n", + "df.aggregate(\n", + " [col(\"bool_col\")],\n", + " [\n", + " f.count_star().alias(\"count\"),\n", + " f.sum(col(\"id\")).alias(\"sum_id\"),\n", + " f.avg(col(\"id\")).alias(\"avg_id\"),\n", + " ]\n", + ")" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Distinct Values" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Get distinct values\n", + "df.select(\"bool_col\").distinct()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Chaining Operations\n", + "\n", + "DataFrame operations can be chained together to build complex transformations." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Complex chained query\n", + "result = (\n", + " ctx.table(\"test_data\")\n", + " .select(\"id\", \"bool_col\", \"tinyint_col\")\n", + " .filter(col(\"id\") > lit(2))\n", + " .sort(col(\"id\").sort(ascending=False))\n", + " .limit(5)\n", + ")\n", + "\n", + "result" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# View the execution plan for the chained query\n", + "print(result.explain())" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Visual execution plan\n", + "result.explain_visual()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Collecting Results" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Collect as Arrow batches\n", + "batches = result.collect()\n", + "print(f\"Got {len(batches)} batch(es)\")\n", + "print(f\"Total rows: {sum(len(batch) for batch in batches)}\")" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Collect as Arrow table\n", + "table = result.to_arrow_table()\n", + "print(f\"Arrow table: {table.num_rows} rows, {table.num_columns} columns\")" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Convert to Pandas\n", + "pdf = result.to_pandas()\n", + "pdf" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Get the count without collecting all data\n", + "count = ctx.table(\"test_data\").count()\n", + "print(f\"Total rows in test_data: {count}\")" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Working with CSV Data" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Read CSV directly without registering\n", + "csv_df = ctx.read_csv(\"../testdata/test.csv\", has_header=True)\n", + "\n", + "# Show schema and data\n", + "print(\"CSV Schema:\")\n", + "for field in csv_df.schema():\n", + " print(f\" {field.name}: {field.type}\")\n", + "\n", + "csv_df" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Filter CSV data\n", + "csv_df.filter(col(\"a\") > lit(2))" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Next Steps\n", + "\n", + "- See `distributed_queries.ipynb` for examples of distributed query execution\n", + "- Check the [DataFusion Python documentation](https://datafusion.apache.org/python/) for more DataFrame operations\n", + "- Review the SQL magic commands in `getting_started.ipynb` for interactive querying" + ] + } + ], + "metadata": { + "kernelspec": { + "display_name": "Python 3", + "language": "python", + "name": "python3" + }, + "language_info": { + "codemirror_mode": { + "name": "ipython", + "version": 3 + }, + "file_extension": ".py", + "mimetype": "text/x-python", + "name": "python", + "nbconvert_exporter": "python", + "pygments_lexer": "ipython3", + "version": "3.11.0" + } + }, + "nbformat": 4, + "nbformat_minor": 4 +} diff --git a/python/examples/distributed_queries.ipynb b/python/examples/distributed_queries.ipynb new file mode 100644 index 000000000..2c3c14902 --- /dev/null +++ b/python/examples/distributed_queries.ipynb @@ -0,0 +1,411 @@ +{ + "cells": [ + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "\n", + "\n", + "# Distributed Queries with Ballista\n", + "\n", + "This notebook demonstrates distributed query execution features in Ballista.\n", + "\n", + "## Overview\n", + "\n", + "Ballista is a distributed query engine that can execute queries across multiple\n", + "nodes. When you submit a query, Ballista:\n", + "\n", + "1. Parses and optimizes the query\n", + "2. Creates a distributed execution plan\n", + "3. Distributes work across executors\n", + "4. Collects and returns results\n", + "\n", + "This enables processing of datasets much larger than a single machine's memory." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "from ballista import BallistaSessionContext, setup_test_cluster\n", + "from datafusion import col, lit\n", + "from datafusion import functions as f\n", + "\n", + "# Set up test cluster and connect\n", + "host, port = setup_test_cluster()\n", + "ctx = BallistaSessionContext(f\"df://{host}:{port}\")\n", + "\n", + "# Register sample data\n", + "ctx.register_parquet(\"test_data\", \"../testdata/test.parquet\")\n", + "\n", + "print(f\"Connected! Session ID: {ctx.session_id}\")" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Execution Plans\n", + "\n", + "Understanding execution plans is key to optimizing distributed queries." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Create a query with multiple stages\n", + "df = ctx.sql(\"\"\"\n", + " SELECT \n", + " bool_col,\n", + " COUNT(*) as cnt,\n", + " SUM(id) as sum_id,\n", + " AVG(tinyint_col) as avg_tiny\n", + " FROM test_data\n", + " WHERE id > 2\n", + " GROUP BY bool_col\n", + " ORDER BY cnt DESC\n", + "\"\"\")\n", + "\n", + "df" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# View the logical plan\n", + "print(\"Logical Plan:\")\n", + "print(df.explain())" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Visualize the execution plan\n", + "# This shows the query plan as a graph (requires graphviz for full SVG)\n", + "df.explain_visual()" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# View the plan with runtime statistics (analyze=True runs the query)\n", + "print(\"Analyzed Plan (with statistics):\")\n", + "print(df.explain(analyze=True))" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Multi-Stage Queries\n", + "\n", + "Complex queries may involve multiple stages of distributed execution." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Subquery example\n", + "result = ctx.sql(\"\"\"\n", + " WITH stats AS (\n", + " SELECT \n", + " bool_col,\n", + " COUNT(*) as cnt\n", + " FROM test_data\n", + " GROUP BY bool_col\n", + " )\n", + " SELECT \n", + " t.id,\n", + " t.bool_col,\n", + " s.cnt as group_count\n", + " FROM test_data t\n", + " JOIN stats s ON t.bool_col = s.bool_col\n", + " WHERE t.id <= 5\n", + " ORDER BY t.id\n", + "\"\"\")\n", + "\n", + "result" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# View the execution plan - notice the join and exchange stages\n", + "result.explain_visual()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## DataFrame API for Complex Transformations" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Build a complex transformation using the DataFrame API\n", + "df1 = ctx.table(\"test_data\")\n", + "\n", + "# Aggregate to get group statistics\n", + "group_stats = df1.aggregate(\n", + " [col(\"bool_col\")],\n", + " [\n", + " f.count_star().alias(\"group_count\"),\n", + " f.avg(col(\"id\")).alias(\"avg_id\"),\n", + " ]\n", + ")\n", + "\n", + "group_stats" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Join original data with statistics\n", + "joined = df1.join(\n", + " group_stats,\n", + " on=\"bool_col\",\n", + " how=\"inner\"\n", + ")\n", + "\n", + "joined.select(\"id\", \"bool_col\", \"group_count\", \"avg_id\").limit(10)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Window Functions\n", + "\n", + "Window functions allow computations across related rows." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Window function example\n", + "window_result = ctx.sql(\"\"\"\n", + " SELECT \n", + " id,\n", + " bool_col,\n", + " tinyint_col,\n", + " SUM(tinyint_col) OVER (\n", + " PARTITION BY bool_col \n", + " ORDER BY id\n", + " ) as running_sum,\n", + " ROW_NUMBER() OVER (\n", + " PARTITION BY bool_col \n", + " ORDER BY id\n", + " ) as row_num\n", + " FROM test_data\n", + " ORDER BY bool_col, id\n", + "\"\"\")\n", + "\n", + "window_result" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Progress Tracking for Long Queries\n", + "\n", + "For long-running queries, you can track progress." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Execute a query with progress tracking\n", + "df = ctx.sql(\"SELECT * FROM test_data\")\n", + "\n", + "# collect_with_progress shows elapsed time in Jupyter\n", + "batches = df.collect_with_progress()\n", + "print(f\"Collected {len(batches)} batch(es)\")" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# You can also provide a custom callback\n", + "def my_progress_callback(status, progress):\n", + " if progress < 0:\n", + " print(f\"Status: {status} (in progress...)\")\n", + " else:\n", + " print(f\"Status: {status} ({progress:.0%} complete)\")\n", + "\n", + "df = ctx.sql(\"SELECT * FROM test_data\")\n", + "batches = df.collect_with_progress(callback=my_progress_callback)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Writing Results\n", + "\n", + "Distributed write operations for large result sets." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Prepare a query result\n", + "df = ctx.sql(\"\"\"\n", + " SELECT \n", + " id,\n", + " bool_col,\n", + " tinyint_col * 2 as doubled\n", + " FROM test_data\n", + " WHERE id > 3\n", + "\"\"\")\n", + "\n", + "df" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Write to Parquet (distributed write)\n", + "# df.write_parquet(\"../target/output.parquet\")\n", + "\n", + "# Write to CSV\n", + "# df.write_csv(\"../target/output.csv\")\n", + "\n", + "# Write to JSON\n", + "# df.write_json(\"../target/output.json\")\n", + "\n", + "print(\"Write operations are commented out - uncomment to test\")" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Best Practices for Distributed Queries\n", + "\n", + "1. **Filter early**: Push filters as close to the data source as possible\n", + "2. **Project early**: Select only needed columns to reduce data movement\n", + "3. **Partition wisely**: Ensure data is partitioned for efficient joins\n", + "4. **Check plans**: Use `explain()` and `explain_visual()` to understand execution\n", + "5. **Monitor progress**: Use `collect_with_progress()` for long queries" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Example: Optimized query pattern\n", + "optimized = (\n", + " ctx.table(\"test_data\")\n", + " # 1. Filter early\n", + " .filter(col(\"id\") > lit(2))\n", + " # 2. Project only needed columns\n", + " .select(\"id\", \"bool_col\", \"tinyint_col\")\n", + " # 3. Aggregate\n", + " .aggregate(\n", + " [col(\"bool_col\")],\n", + " [f.count_star().alias(\"cnt\")]\n", + " )\n", + ")\n", + "\n", + "# 4. Check the plan\n", + "print(\"Optimized plan:\")\n", + "print(optimized.explain())" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Next Steps\n", + "\n", + "- Review the [Ballista Architecture docs](https://datafusion.apache.org/ballista/)\n", + "- Learn about cluster deployment and configuration\n", + "- Explore advanced features like custom functions and plugins" + ] + } + ], + "metadata": { + "kernelspec": { + "display_name": "Python 3", + "language": "python", + "name": "python3" + }, + "language_info": { + "codemirror_mode": { + "name": "ipython", + "version": 3 + }, + "file_extension": ".py", + "mimetype": "text/x-python", + "name": "python", + "nbconvert_exporter": "python", + "pygments_lexer": "ipython3", + "version": "3.11.0" + } + }, + "nbformat": 4, + "nbformat_minor": 4 +} diff --git a/python/examples/getting_started.ipynb b/python/examples/getting_started.ipynb new file mode 100644 index 000000000..b2f725cba --- /dev/null +++ b/python/examples/getting_started.ipynb @@ -0,0 +1,329 @@ +{ + "cells": [ + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "\n", + "\n", + "# Getting Started with PyBallista\n", + "\n", + "This notebook demonstrates how to get started with Ballista using Python.\n", + "\n", + "## Prerequisites\n", + "\n", + "1. Install PyBallista: `pip install ballista`\n", + "2. Have a Ballista cluster running (or use the built-in test cluster)\n", + "\n", + "## Overview\n", + "\n", + "Ballista is a distributed query engine built on Apache DataFusion. PyBallista provides:\n", + "\n", + "- **BallistaSessionContext**: Drop-in replacement for DataFusion's SessionContext\n", + "- **SQL Magic Commands**: Interactive SQL in Jupyter notebooks via `%sql` and `%%sql`\n", + "- **DataFrame API**: Full DataFrame API for data transformations\n", + "- **Rich HTML Display**: DataFrames render as styled HTML tables" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Method 1: Python API\n", + "\n", + "The most straightforward way to use Ballista is via the Python API." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import ballista\n", + "from ballista import BallistaSessionContext, setup_test_cluster\n", + "%load_ext autoreload\n", + "%autoreload 2\n", + "\n", + "# Check versions\n", + "print(f\"Ballista version: {ballista.__version__}\")" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# For this demo, we'll use the built-in test cluster\n", + "# In production, you would connect to your Ballista scheduler:\n", + "# ctx = BallistaSessionContext(\"df://your-scheduler:50050\")\n", + "\n", + "host, port = setup_test_cluster()\n", + "ctx = BallistaSessionContext(f\"df://{host}:{port}\")\n", + "\n", + "print(f\"Connected to Ballista at {host}:{port}\")\n", + "print(f\"Session ID: {ctx.session_id}\")" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Register two .parquet files at the same db schema\n", + "ctx.register_parquet(\"public.test_data_v1\", \"../testdata/test.parquet\")\n", + "ctx.register_parquet(\"public.test_data_v2\", \"../testdata/test.parquet\")\n", + "\n", + "# List registered tables\n", + "print(\"Registered schemas and tables:\\n\", ctx.get_tables())" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Execute a SQL query - the DataFrame will render as a nice HTML table\n", + "df = ctx.sql(\"SELECT * FROM public.test_data_v1 LIMIT 10\")\n", + "df" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# You can also use show() for terminal-style output\n", + "df.show(5)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Get the execution plan\n", + "print(df.explain())" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Visualize the execution plan (requires graphviz for full SVG)\n", + "df.explain_visual()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Method 2: SQL Magic Commands\n", + "\n", + "For a more interactive experience, use the SQL magic commands!" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Load the Ballista Jupyter extension\n", + "%reload_ext ballista.jupyter" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Connect to the cluster\n", + "%ballista connect df://localhost:39431" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Register .parquet table\n", + "%register parquet public.test_data_v1 ../testdata/test.parquet" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Check connection status\n", + "%ballista status" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# List registered tables\n", + "%ballista tables" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Execute a single-line SQL query\n", + "%sql select count(*) as count_d from public.test_data_v1" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "%%sql\n", + "-- Multi-line queries work with %%sql cell magic\n", + "SELECT\n", + " id,\n", + " bool_col,\n", + " tinyint_col\n", + "FROM test_data_v1\n", + "WHERE id > 2\n", + "ORDER BY id\n", + "LIMIT 5" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "%%sql my_result\n", + "-- Store the result in a variable for further processing\n", + "SELECT * FROM test_data_v1 WHERE id <= 3" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# View query history\n", + "%ballista history" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "%ballista help" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Data Export\n", + "\n", + "Ballista supports exporting data in multiple formats." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "df = ctx.sql(\"SELECT * FROM test_data_v1 LIMIT 100\")\n", + "\n", + "# Export to various formats\n", + "# df.write_parquet(\"output.parquet\")\n", + "# df.write_csv(\"output.csv\")\n", + "# df.write_json(\"output.json\")\n", + "\n", + "# Convert to Arrow, Pandas, or Polars\n", + "arrow_table = df.to_arrow_table()\n", + "print(f\"Arrow Table Schema:\\n{arrow_table.schema}\")" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Convert to pandas\n", + "pandas_df = df.to_pandas()\n", + "pandas_df.head()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Next Steps\n", + "\n", + "- Check out the `dataframe_api.ipynb` notebook for more DataFrame operations\n", + "- See `distributed_queries.ipynb` for examples of distributed query execution\n", + "- Read the [PyBallista documentation](https://datafusion.apache.org/ballista/) for more details" + ] + } + ], + "metadata": { + "kernelspec": { + "display_name": "Python 3 (ipykernel)", + "language": "python", + "name": "python3" + }, + "language_info": { + "codemirror_mode": { + "name": "ipython", + "version": 3 + }, + "file_extension": ".py", + "mimetype": "text/x-python", + "name": "python", + "nbconvert_exporter": "python", + "pygments_lexer": "ipython3", + "version": "3.12.3" + } + }, + "nbformat": 4, + "nbformat_minor": 4 +} diff --git a/python/pyproject.toml b/python/pyproject.toml index bce9f5d11..84f36c7eb 100644 --- a/python/pyproject.toml +++ b/python/pyproject.toml @@ -46,6 +46,11 @@ dependencies = [ ] dynamic = ["version"] +[project.optional-dependencies] +jupyter = [ + "ipython>=8.0.0", +] + [project.urls] homepage = "https://datafusion.apache.org/ballista" documentation = "https://datafusion.apache.org/ballista" diff --git a/python/python/ballista/__init__.py b/python/python/ballista/__init__.py index 0c3aaa75a..5a5ef7bb0 100644 --- a/python/python/ballista/__init__.py +++ b/python/python/ballista/__init__.py @@ -25,7 +25,11 @@ BallistaExecutor, setup_test_cluster, ) -from .extension import BallistaSessionContext +from .extension import ( + BallistaSessionContext, + DistributedDataFrame, + ExecutionPlanVisualization, +) __version__ = importlib_metadata.version(__name__) @@ -34,4 +38,6 @@ "BallistaScheduler", "BallistaExecutor", "BallistaSessionContext", + "DistributedDataFrame", + "ExecutionPlanVisualization", ] diff --git a/python/python/ballista/extension.py b/python/python/ballista/extension.py index e1217e845..87f2d6d66 100644 --- a/python/python/ballista/extension.py +++ b/python/python/ballista/extension.py @@ -19,8 +19,12 @@ from datafusion.dataframe import Compression from typing import ( + List, Union, + Optional, + Callable ) +import warnings from ._internal_ballista import create_ballista_data_frame @@ -28,26 +32,6 @@ from ._internal_ballista import ParquetWriterOptions as ParquetWriterOptionsInternal import pathlib -# DataFrame execution methods which should be automatically -# overridden. - -OVERRIDDEN_EXECUTION_METHODS = [ - "show", - "count", - "collect", - "collect_partitioned", - "write_json", - "to_arrow_table", - "to_pandas", - "to_pydict", - "to_polars", - "to_pylist", - "_repr_html_", - "execute_stream", - "execute_stream_partitioned", -] - - # class used to redefine DataFrame object # intercepting execution methods and methods # which returns `DataFrame` @@ -90,10 +74,6 @@ def method_wrapper(*args, **kwargs): # attrs[base_name] = __wrap_dataframe_result(base_value) - # TODO: we could do better here - for function in OVERRIDDEN_EXECUTION_METHODS: - attrs[function] = __wrap_dataframe_execution(function) - return super().__new__(cls, name, bases, attrs) @@ -131,21 +111,18 @@ def method_wrapper(*args, **kwargs): # serialize it and invoke ballista client to execute it # # this class keeps reference to remote ballista - - -class DistributedDataFrame(DataFrame, metaclass=RedefiningDataFrameMeta): +class DistributedDataFrame(DataFrame, metaclass=type): def __init__(self, df: DataFrame, session_id: str, address: str): super().__init__(df.df) self.address = address - self.session_id = session_id - + self._session_id = session_id # # this will create a ballista dataframe, which has ballista # session context, and ballista planner. # def _to_internal_df(self): blob_plan = self.logical_plan().to_proto() - df = create_ballista_data_frame(blob_plan, self.address, self.session_id) + df = create_ballista_data_frame(blob_plan, self.address, self._session_id) return df def write_csv(self, path, with_header=False): @@ -229,9 +206,325 @@ def write_parquet( df = self._to_internal_df() df.write_parquet(str(path), compression.value, compression_level) + def explain_visual(self, analyze: bool = False) -> "ExecutionPlanVisualization": + """ + Generate a visual representation of the execution plan. + + This method creates an SVG visualization of the query execution plan, + which can be displayed directly in Jupyter notebooks. + + Args: + analyze: If True, includes runtime statistics from actual execution. + + Returns: + ExecutionPlanVisualization: An object that renders as SVG in Jupyter. + + Example: + >>> df = ctx.sql("SELECT * FROM orders WHERE amount > 100") + >>> df.explain_visual() # Displays SVG in notebook + >>> viz = df.explain_visual(analyze=True) + >>> viz.save("plan.svg") # Save to file + """ + # Get the execution plan as a string representation + # Note: explain() prints but doesn't return a string, so we use logical_plan() + try: + plan = self.logical_plan() + plan_str = plan.display_indent() + except Exception: + # Fallback if logical_plan() fails + plan_str = "Unable to retrieve execution plan" + return ExecutionPlanVisualization(plan_str, analyze=analyze) + + def collect_with_progress( + self, + callback: Optional[Callable] = None, + poll_interval: float = 0.5, + ): + """ + Collect results with progress indication. + + For long-running queries, this method provides progress updates + through a callback function or displays a progress bar in Jupyter. + + Args: + callback: Optional function to call with progress updates. + Signature: callback(status: str, progress: float) + poll_interval: How often to check progress (seconds). + + Returns: + The collected result batches. + + Example: + >>> def my_callback(status, progress): + ... print(f"{status}: {progress:.1%}") + >>> batches = df.collect_with_progress(callback=my_callback) + """ + import threading + import time + + result = [None] + error = [None] + done = threading.Event() + + def execute(): + try: + result[0] = self.collect() + except Exception as e: + error[0] = e + finally: + done.set() + + thread = threading.Thread(target=execute) + thread.start() + + # Check if we're in a Jupyter environment + try: + from IPython.display import clear_output + from IPython.core.getipython import get_ipython + + in_jupyter = get_ipython() is not None + except (ImportError, AttributeError): + in_jupyter = False + + start_time = time.time() + + if in_jupyter and callback is None: + # Display a simple progress indicator + try: + while not done.wait(timeout=poll_interval): + elapsed = time.time() - start_time + clear_output(wait=True) + print(f"⏳ Query executing... ({elapsed:.1f}s elapsed)") + + clear_output(wait=True) + elapsed = time.time() - start_time + print(f"✓ Query completed in {elapsed:.1f}s") + except Exception: + pass # Ignore display errors + elif callback is not None: + while not done.wait(timeout=poll_interval): + elapsed = time.time() - start_time + callback(f"Executing ({elapsed:.1f}s)", -1.0) # -1 means indeterminate + + elapsed = time.time() - start_time + callback(f"Completed in {elapsed:.1f}s", 1.0) + else: + done.wait() + + thread.join() + + if error[0] is not None: + raise error[0] + + return result[0] + + +class ExecutionPlanVisualization: + """ + A wrapper for execution plan visualizations that can render as SVG in Jupyter. + + This class takes the text representation of an execution plan and converts + it to a Graphviz DOT format, which is then rendered as SVG. + """ + + def __init__(self, plan_str: str, analyze: bool = False): + self.plan_str = plan_str + self.analyze = analyze + self._svg_cache: Optional[str] = None + + def _parse_plan_to_dot(self) -> str: + """Convert the plan string to DOT format for Graphviz.""" + lines = self.plan_str.strip().split("\n") + + dot_lines = [ + "digraph ExecutionPlan {", + ' rankdir=TB;', + ' node [shape=box, style="rounded,filled", fontname="Helvetica"];', + ' edge [fontname="Helvetica"];', + "", + ] + + nodes = [] + edges = [] + node_id = 0 + stack = [] # (indent_level, node_id) + + for line in lines: + if not line.strip(): + continue + + # Calculate indent level + indent = len(line) - len(line.lstrip()) + content = line.strip() + + # Skip non-plan lines + if content.startswith("physical_plan") or content.startswith("logical_plan"): + continue + + # Create a node for this plan element + current_id = node_id + node_id += 1 + + # Determine node color based on operation type + color = "#E3F2FD" # Default light blue + if "Scan" in content or "TableScan" in content: + color = "#E8F5E9" # Light green for scans + elif "Filter" in content: + color = "#FFF3E0" # Light orange for filters + elif "Aggregate" in content or "HashAggregate" in content: + color = "#F3E5F5" # Light purple for aggregations + elif "Join" in content: + color = "#FFEBEE" # Light red for joins + elif "Sort" in content: + color = "#E0F7FA" # Light cyan for sorts + elif "Projection" in content: + color = "#FFF8E1" # Light amber for projections + + # Escape special characters for DOT format + label = content.replace('"', '\\"').replace("\n", "\\n") + if len(label) > 60: + # Wrap long labels + label = label[:57] + "..." + + nodes.append(f' node{current_id} [label="{label}", fillcolor="{color}"];') + + # Connect to parent based on indentation + while stack and stack[-1][0] >= indent: + stack.pop() + + if stack: + parent_id = stack[-1][1] + edges.append(f" node{parent_id} -> node{current_id};") + + stack.append((indent, current_id)) + + dot_lines.extend(nodes) + dot_lines.append("") + dot_lines.extend(edges) + dot_lines.append("}") + + return "\n".join(dot_lines) + + def to_dot(self) -> str: + """Get the DOT representation of the execution plan.""" + return self._parse_plan_to_dot() + + def to_svg(self) -> str: + """ + Convert the execution plan to SVG format. + + Requires graphviz to be installed. If graphviz is not available, + returns a simple HTML representation instead. + """ + if self._svg_cache is not None: + return self._svg_cache + + dot_source = self._parse_plan_to_dot() + + try: + import subprocess + + # Try to use graphviz's dot command + process = subprocess.run( + ["dot", "-Tsvg"], + input=dot_source.encode(), + capture_output=True, + timeout=30, + ) + + if process.returncode == 0: + self._svg_cache = process.stdout.decode() + return self._svg_cache + except (subprocess.SubprocessError, FileNotFoundError, subprocess.TimeoutExpired) as e: + warnings.warn(f"Could not convert the execution plan to SVG format: {e}") + pass + + # Fallback: return a pre-formatted HTML representation + escaped_plan = ( + self.plan_str.replace("&", "&") + .replace("<", "<") + .replace(">", ">") + ) + self._svg_cache = f""" +
+
+ Execution Plan {'(with statistics)' if self.analyze else ''} +
Install graphviz for visual diagram: brew install graphviz +
+
{escaped_plan}
+
+ """ + return self._svg_cache + + def save(self, path: str) -> None: + """Save the visualization to a file (SVG or DOT format).""" + if path.endswith(".dot"): + content = self.to_dot() + else: + content = self.to_svg() + + with open(path, "w") as f: + f.write(content) + + def _repr_html_(self) -> str: + """HTML representation for Jupyter notebooks.""" + return self.to_svg() + + def _repr_svg_(self) -> str: + """SVG representation for Jupyter notebooks.""" + svg = self.to_svg() + # Only return if it's actual SVG content + if svg.strip().startswith(" str: + """String representation.""" + return f"ExecutionPlanVisualization(analyze={self.analyze})\n{self.plan_str}" + class BallistaSessionContext(SessionContext, metaclass=RedefiningSessionContextMeta): + """ + A session context for connecting to and querying a Ballista cluster. + + This class extends DataFusion's SessionContext to work with distributed + Ballista clusters, automatically routing query execution to the cluster + while maintaining API compatibility with local DataFusion usage. + + Example: + >>> from ballista import BallistaSessionContext + >>> ctx = BallistaSessionContext("df://localhost:50050") + >>> df = ctx.sql("SELECT * FROM my_table LIMIT 10") + >>> df.show() + + For Jupyter notebook users: + >>> %load_ext ballista.jupyter + >>> %ballista connect df://localhost:50050 + >>> %sql SELECT * FROM my_table + """ + def __init__(self, address: str, config=None, runtime=None): super().__init__(config, runtime) self.address = address - self.session_id = self.session_id() + self.session_id_internal = super().session_id() + + @property + def session_id(self): + return self.session_id_internal + + def get_tables(self) -> Optional[dict[str, List[str]]]: + """Get tables and their respective schemas (in terms of database schema).""" + try: + catalog = self.catalog() + schema_names = list(catalog.schema_names()) + if schema_names: + tables_info = {} + for schema_name in schema_names: + tables_info[schema_name] = list(catalog.schema(name=schema_name).table_names()) + return tables_info + except (AttributeError, NotImplementedError) as e: + warnings.warn(f"Could not retrieve tables from catalog: {e}") + pass + return {} + \ No newline at end of file diff --git a/python/python/ballista/jupyter.py b/python/python/ballista/jupyter.py new file mode 100644 index 000000000..c6e2b83de --- /dev/null +++ b/python/python/ballista/jupyter.py @@ -0,0 +1,494 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +""" +Ballista Jupyter Magic Commands. + +This module provides IPython magic commands for interacting with Ballista +clusters directly from Jupyter notebooks. + +Usage: + %load_ext ballista.jupyter + + # Connect to a Ballista cluster + %ballista connect df://localhost:50050 + + # Check connection status + %ballista status + + # List registered tables + %ballista tables + + # Show schema for a table + %ballista schema my_table + + # Execute a simple SQL query (line magic) + %sql SELECT COUNT(*) FROM my_table + + # Execute a complex SQL query (cell magic) + %%sql + SELECT + customer_id, + SUM(amount) as total + FROM orders + GROUP BY customer_id + ORDER BY total DESC + LIMIT 10 +""" + +from typing import Optional, List, Dict, Any +import warnings +import time + +try: + from IPython.core.magic import Magics, magics_class, line_magic, line_cell_magic + from IPython.display import display, HTML + + IPYTHON_AVAILABLE = True +except ImportError: + IPYTHON_AVAILABLE = False + + # Provide stub classes for when IPython is not available + class Magics: + def __init__(self, shell=None): + self.shell = shell + + def magics_class(cls): + return cls + + def line_magic(name_or_func): + """Stub line_magic decorator for when IPython is not available.""" + # Handle both @line_magic and @line_magic("name") usage + if callable(name_or_func): + # Used as @line_magic without arguments + return name_or_func + else: + # Used as @line_magic("name") with arguments + def decorator(func): + return func + return decorator + + def cell_magic(name_or_func): + """Stub cell_magic decorator for when IPython is not available.""" + # Handle both @cell_magic and @cell_magic("name") usage + if callable(name_or_func): + return name_or_func + else: + def decorator(func): + return func + return decorator + + def line_cell_magic(name_or_func): + """Stub line_cell_magic decorator for when IPython is not available""" + # Handle both @line_cell_magic and @line_cell_magic("name") usage + if callable(name_or_func): + return name_or_func + else: + def decorator(func): + return func + return decorator + + +from .extension import BallistaSessionContext, DistributedDataFrame + + +class BallistaConnectionError(Exception): + """Raised when not connected to a Ballista cluster.""" + + pass + + +@magics_class +class BallistaMagics(Magics): + """ + IPython magic commands for Ballista. + + Provides convenient commands for connecting to Ballista clusters, + executing SQL queries, and exploring table schemas. + """ + + def __init__(self, shell=None): + super().__init__(shell) + self._ctx: Optional[BallistaSessionContext] = None + self._address: Optional[str] = None + self._last_result: Optional[DistributedDataFrame] = None + self._query_history: List[Dict[str, Any]] = [] + + @property + def ctx(self) -> BallistaSessionContext: + """Get the current context, raising an error if not connected.""" + if self._ctx is None: + raise BallistaConnectionError( + "Not connected to a Ballista cluster. " + "Use: %ballista connect df://host:port" + ) + return self._ctx + + @property + def is_connected(self) -> bool: + """Check if connected to a Ballista cluster.""" + return self._ctx is not None + + @line_magic + def ballista(self, line: str) -> Optional[str]: + """ + Ballista management commands. + + Usage: + %ballista connect df://localhost:50050 - Connect to cluster + %ballista status - Show connection status + %ballista tables - List registered tables + %ballista schema - Show table schema + %ballista disconnect - Disconnect from cluster + %ballista history - Show query history + + Examples: + %ballista connect df://localhost:50050 + %ballista tables + %ballista schema orders + """ + parts = line.strip().split(maxsplit=1) + if not parts: + return self._show_help() + + cmd = parts[0].lower() + args = parts[1] if len(parts) > 1 else "" + + if cmd == "connect": + return self._connect(args) + elif cmd == "status": + return self._status() + elif cmd == "tables": + return self._tables() + elif cmd == "schema": + return self._schema(args) + elif cmd == "disconnect": + return self._disconnect() + elif cmd == "history": + return self._show_history() + elif cmd == "help": + return self._show_help() + else: + return f"Unknown command: {cmd}. Use '%ballista help' for available commands." + + @line_magic + def register(self, line: str) -> Optional[str]: + """Register a new table""" + if not line: + return "You should provide file extension and table name to register" + elif self._ctx is None: + raise BallistaConnectionError( + "Not connected to a Ballista cluster. " + "Use: %ballista connect df://host:port" + ) + else: + args = line.strip().split() + file_type = args[0] + if len(args) < 2: + return f"You should provide table name for this .{file_type} file" + table_name = args[1] + if len(args) < 3: + return "You should provide path to your file" + file_name = args[2] + + if file_type == "parquet": + self._ctx.register_parquet(table_name, file_name) + elif file_type == "csv": + self._ctx.register_csv(table_name, file_name) + else: + raise NotImplementedError("Currently not supporting the inserted file format") + + @line_cell_magic + def sql(self, line: str, cell=None) -> Optional[DistributedDataFrame]: + """ + Execute a SQL query (both line and cell magic). + + Two cases possible: with cell or without cell + + Examples: + 1. Without a cell (line_magic): + %sql SELECT * FROM test_table + 2. With a cell (cell_magic) -- we can also use a variable to store the result of the query like this: + %%sql my_result + SELECT + id, + bool_col, + tinyint_col + FROM test_data_v1 + WHERE id > 2 + ORDER BY id + LIMIT 5 + + `my_result` will store the result of the SQL-query + """ + if not cell: + return self._execute_sql(line.strip()) if line.strip() else None + else: + var_name = None + query = cell.strip() + if not query: + return None + + args = line.strip().split() + i = 0 + while i < len(args): + if not args[i].startswith("--"): + var_name = args[i] + i += 1 + + result = self._execute_sql(query) + + # Store in user namespace if variable name provided + if var_name and self.shell is not None: + self.shell.user_ns[var_name] = result + return result + + def _connect(self, address: str) -> Optional[str]: + """Connect to a Ballista cluster.""" + if not address: + return "Usage: %ballista connect df://host:port" + + # Normalize address + if not address.startswith("df://"): + address = f"df://{address}" + + try: + self._ctx = BallistaSessionContext(address) + self._address = address + if IPYTHON_AVAILABLE: + display(HTML(f"✓ Connected to Ballista cluster at {address}")) + else: + print(f"✓ Connected to Ballista cluster at {address}") + except Exception as e: + self._ctx = None + self._address = None + if IPYTHON_AVAILABLE: + display(HTML(f"✗ Failed to connect to {address}: {e}")) + else: + print(f"✗ Failed to connect to {address}: {e}") + + def _disconnect(self) -> Optional[str]: + """Disconnect from the Ballista cluster.""" + if not self.is_connected: + return "Not connected to any cluster." + + address = self._address + self._ctx = None + self._address = None + self._last_result = None + if IPYTHON_AVAILABLE: + display(HTML(f"✓ Disconnected from {address}")) + else: + print(f"✓ Disconnected from {address}") + + def _status(self) -> Optional[str]: + """Show connection status.""" + if not self.is_connected: + return "Status: Not connected\n\nUse '%ballista connect df://host:port' to connect." + + status_lines = [ + "Status: Connected", + f"Address: {self._address}", + f"Session ID: {self._ctx.session_id}", + f"Queries executed: {len(self._query_history)}", + ] + + if self._last_result is not None: + status_lines.append("Last result: Available (access via '_' or '_last_result')") + + def _format_html_status_output(line: str) -> str: + name, value = line.split(":", 1) + return f"
{name}: {value.strip()}
" + + html = "".join(_format_html_status_output(line) for line in status_lines) + if IPYTHON_AVAILABLE: + display(HTML(html)) + else: + print("\n".join(status_lines)) + + def _tables(self) -> Optional[str]: + """List all registered tables.""" + try: + # Get table names from the catalog and their respective schemas + tables = self.ctx.get_tables() + if not tables: + return "No tables registered.\n\nUse ctx.register_parquet() or ctx.register_csv() to register tables." + schema_count = len(tables.keys()) + table_count = sum(len(v) for v in tables.values()) + # Build a nice table display (HTML-formatted if applicable) + lines = [ + {"content": f"Total: {table_count} table(s) in {schema_count} schema(s)", "is_info": True}, + {"content": "Registered tables:", "is_info": True}, + *[{"content": f"Schema: {schema_name}. Tables: {', '.join(table_names)}", "is_info": False} + for schema_name, table_names in tables.items()] + ] + + def _format_html_tables_output(line: str, is_info: bool = False) -> str: + if is_info: + return f"
{line}
" + else: + return f"

{line}

" + + if IPYTHON_AVAILABLE: + display( + HTML("".join(_format_html_tables_output(val["content"], val["is_info"]) for val in lines)) + ) + else: + print("".join(val["content"] for val in lines)) + except Exception as e: + warnings.warn(f"Error listing tables: {e}") + + def _schema(self, table_name: str) -> Optional[str]: + """Show schema for a table.""" + if not table_name: + return "Usage: %ballista schema " + + try: + # Query the table with LIMIT 0 to get schema without data + df = self.ctx.sql(f"SELECT * FROM {table_name} LIMIT 0") + schema = df.schema() + + lines = [f"Schema for '{table_name}':", "-" * 50] + for field in schema: + nullable = "NULL" if field.nullable else "NOT NULL" + lines.append(f" {field.name:20} {str(field.type):15} {nullable}") + lines.append("-" * 50) + lines.append(f"Total: {len(schema)} column(s)") + + print("\n".join(lines)) + except Exception as e: + warnings.warn(f"Error getting schema for '{table_name}': {e}") + + def _execute_sql( + self, + query: str, + ) -> Optional[DistributedDataFrame]: + """Execute a SQL query and return the result.""" + start_time = time.time() + + try: + result = self.ctx.sql(query) + elapsed = time.time() - start_time + + # Store result + self._last_result = result + if self.shell is not None and hasattr(self.shell, 'user_ns'): + self.shell.user_ns["_last_result"] = result + + # Record in history + self._query_history.append( + { + "query": query, + "elapsed_seconds": elapsed, + "timestamp": time.strftime("%Y-%m-%d %H:%M:%S"), + } + ) + + # Display if requested and in notebook environment + return result + + except Exception as e: + error_msg = f"Query failed: {e}" + if IPYTHON_AVAILABLE: + try: + display(HTML(f'
{error_msg}
')) + except Exception: + print(error_msg) + else: + print(error_msg) + return None + + def _show_history(self) -> Optional[str]: + """Show query history.""" + if not self._query_history: + return "No queries executed yet." + + lines = ["Query History:", "-" * 60] + for i, entry in enumerate(self._query_history[-10:], 1): # Last 10 queries + query_preview = entry["query"][:50] + "..." if len(entry["query"]) > 50 else entry["query"] + query_preview = query_preview.replace("\n", " ") + lines.append(f"{i}. [{entry['timestamp']}] ({entry['elapsed_seconds']:.2f}s)") + lines.append(f" {query_preview}") + lines.append("-" * 60) + output = "\n".join(lines) + if IPYTHON_AVAILABLE: + display(HTML(f"
{output}
")) + else: + print(output) + + def _show_help(self) -> Optional[str]: + """Show help for Ballista magic commands.""" + help_info = """ +Ballista Jupyter Magic Commands +================================ + +Connection: + %ballista connect - Connect to Ballista cluster + %ballista disconnect - Disconnect from cluster + %ballista status - Show connection status + +Exploration: + %ballista tables - List registered tables + %ballista schema
- Show table schema + +Table-register: + %register [format] [schema.table_name] [file_path] - Register a new table in the current Ballista Context + +Query: + %sql - Execute single-line SQL query + + %%sql [options] [var] - Execute multi-line SQL query + Options: + --no-display - Don't display results + --limit N - Limit displayed rows (default: 50) + var - Store result in variable + +History: + %ballista history - Show recent query history + +Examples: + %ballista connect df://localhost:50050 + %ballista tables + %ballista schema orders + + %sql SELECT COUNT(*) FROM orders + + %%sql my_result + SELECT customer_id, SUM(amount) as total + FROM orders + GROUP BY customer_id + ORDER BY total DESC + LIMIT 10 + """ + if IPYTHON_AVAILABLE: + display(HTML(f"
{help_info}
")) + else: + print(help_info) + +def load_ipython_extension(ipython): + """ + Load the Ballista IPython extension. + + Usage in Jupyter notebook: + %load_ext ballista.jupyter + """ + ipython.register_magics(BallistaMagics) + + +def unload_ipython_extension(ipython): + """Unload the Ballista IPython extension.""" + pass # IPython handles magic cleanup automatically diff --git a/python/python/tests/test_jupyter.py b/python/python/tests/test_jupyter.py new file mode 100644 index 000000000..73c69ff4f --- /dev/null +++ b/python/python/tests/test_jupyter.py @@ -0,0 +1,316 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +"""Tests for the Ballista Jupyter module.""" + +import pytest +from unittest.mock import patch, MagicMock +from ballista import setup_test_cluster +from ballista.jupyter import ( + BallistaMagics, + BallistaConnectionError, + load_ipython_extension, + IPYTHON_AVAILABLE, +) + + +@pytest.fixture +def magics(): + """Create a BallistaMagics instance for testing.""" + return BallistaMagics(shell=None) + + +@pytest.fixture +def connected_magics(): + """Create a connected BallistaMagics instance.""" + magics = BallistaMagics(shell=None) + (address, port) = setup_test_cluster() + magics._connect(f"df://{address}:{port}") + return magics + + +class TestBallistaMagicsInitialState: + """Tests for initial (disconnected) state.""" + + def test_initial_state(self, magics): + """Test that magics start disconnected.""" + assert not magics.is_connected + assert magics._ctx is None + assert magics._address is None + + def test_ctx_property_raises_when_disconnected(self, magics): + """Test that accessing ctx raises BallistaConnectionError when not connected.""" + with pytest.raises(BallistaConnectionError): + _ = magics.ctx + + def test_disconnect_when_not_connected_returns_message(self, magics): + result = magics._disconnect() + assert "Not connected" in result + + def test_status_when_disconnected_returns_message(self, magics): + result = magics._status() + assert "Not connected" in result + assert "connect" in result.lower() + + def test_show_history_empty_returns_message(self, magics): + result = magics._show_history() + assert "No queries" in result + + def test_show_help_prints(self, magics, capsys): + """Test that help is printed correctly.""" + with patch("ballista.jupyter.IPYTHON_AVAILABLE", False): + magics._show_help() + captured = capsys.readouterr() + assert "connect" in captured.out.lower() + assert "status" in captured.out.lower() + assert "tables" in captured.out.lower() + assert "sql" in captured.out.lower() + + +class TestBallistaMagicsConnect: + """Tests for connect/disconnect behavior.""" + + def test_connect_requires_address_returns_usage(self, magics): + """Test that connect with empty address returns a usage string.""" + result = magics._connect("") + assert result is not None + assert "Usage:" in result + + def test_connect_with_invalid_address_does_not_raise(self, magics): + """Test connection to invalid address fails gracefully without raising.""" + # Should not raise, but context should not be set + magics._connect("df://invalid-host:99999") + # Connection may be lazy; just assert no unhandled exception occurred + + def test_connect_normalizes_address_without_scheme(self, magics): + """Test that connect prepends df:// if missing.""" + # Use an invalid host so it fails, but we can inspect the stored address + magics._connect("localhost:12345") + # Either connected (unlikely with bad host) or failed cleanly + # Either way, _address if set should have df:// prefix + if magics._address is not None: + assert magics._address.startswith("df://") + + def test_successful_connect_sets_state(self, magics): + """Test that successful connect sets ctx, address, and is_connected.""" + (address, port) = setup_test_cluster() + magics._connect(f"df://{address}:{port}") + assert magics.is_connected + assert magics._ctx is not None + assert magics._address is not None + + def test_disconnect_clears_state(self, connected_magics): + """Test that disconnect clears ctx, address and sets is_connected=False.""" + connected_magics._disconnect() + assert not connected_magics.is_connected + assert connected_magics._ctx is None + assert connected_magics._address is None + assert connected_magics._last_result is None + + def test_disconnect_prints_confirmation(self, connected_magics, capsys): + """Test that disconnect prints confirmation message.""" + with patch("ballista.jupyter.IPYTHON_AVAILABLE", False): + connected_magics._disconnect() + captured = capsys.readouterr() + assert "Disconnected" in captured.out + + +class TestBallistaMagicsCommandParsing: + """Tests for %ballista command dispatch.""" + + def test_ballista_help_command(self, magics, capsys): + """Test that help command prints help text.""" + with patch("ballista.jupyter.IPYTHON_AVAILABLE", False): + magics.ballista("help") + captured = capsys.readouterr() + assert "connect" in captured.out.lower() + + def test_ballista_unknown_command_returns_message(self, magics): + """Test that unknown command returns an error message.""" + result = magics.ballista("unknown_command_xyz") + assert result is not None + assert "Unknown command" in result + + def test_ballista_empty_line_shows_help(self, magics, capsys): + """Test that empty line shows help.""" + with patch("ballista.jupyter.IPYTHON_AVAILABLE", False): + magics.ballista("") + captured = capsys.readouterr() + assert "connect" in captured.out.lower() + + def test_ballista_status_dispatches(self, magics): + result = magics.ballista("status") + assert "Not connected" in result + + def test_ballista_disconnect_when_not_connected(self, magics): + """Test disconnect subcommand when not connected.""" + result = magics.ballista("disconnect") + assert result is not None + assert "Not connected" in result + + +class TestBallistaMagicsConnected: + """Tests for BallistaMagics when connected to a real cluster.""" + + def test_is_connected(self, connected_magics): + """Test that is_connected is True after connect.""" + assert connected_magics.is_connected + assert connected_magics._ctx is not None + + def test_status_when_connected_prints(self, connected_magics, capsys): + """Test status shows connected state via print.""" + with patch("ballista.jupyter.IPYTHON_AVAILABLE", False): + connected_magics._status() + captured = capsys.readouterr() + assert "Connected" in captured.out + assert "Session ID" in captured.out + + def test_tables_empty_does_not_raise(self, connected_magics): + """Test listing tables when none registered does not raise.""" + # Should print or return a message, not raise + connected_magics._tables() # no assertion needed beyond "does not raise" + + def test_execute_sql_returns_dataframe(self, connected_magics): + """Test executing a SQL query returns a DistributedDataFrame.""" + result = connected_magics._execute_sql("SELECT 1 as value") + assert result is not None + + def test_execute_sql_records_history(self, connected_magics): + """Test that executed queries are recorded in history.""" + connected_magics._execute_sql("SELECT 1") + assert len(connected_magics._query_history) == 1 + assert "SELECT 1" in connected_magics._query_history[0]["query"] + + def test_execute_sql_history_has_timestamp_and_elapsed(self, connected_magics): + """Test that history entries contain timing info.""" + connected_magics._execute_sql("SELECT 1") + entry = connected_magics._query_history[0] + assert "timestamp" in entry + assert "elapsed_seconds" in entry + assert isinstance(entry["elapsed_seconds"], float) + + def test_execute_sql_stores_last_result(self, connected_magics): + """Test that _last_result is updated after execution.""" + connected_magics._execute_sql("SELECT 1") + assert connected_magics._last_result is not None + + def test_show_history_prints_entries(self, connected_magics, capsys): + """Test that history is printed after queries are executed.""" + connected_magics._execute_sql("SELECT 1") + with patch("ballista.jupyter.IPYTHON_AVAILABLE", False): + connected_magics._show_history() + captured = capsys.readouterr() + assert "SELECT 1" in captured.out + + def test_sql_magic_line_returns_dataframe(self, connected_magics): + """Test %sql line magic returns a DistributedDataFrame.""" + result = connected_magics.sql("SELECT 1 as value") + assert result is not None + + def test_sql_magic_cell_returns_dataframe(self, connected_magics): + """Test %%sql cell magic returns a DistributedDataFrame.""" + result = connected_magics.sql("", cell="SELECT 1 as value") + assert result is not None + + def test_sql_magic_cell_stores_in_shell_namespace(self, connected_magics): + """Test %%sql stores result in shell namespace when var name given.""" + mock_shell = MagicMock() + mock_shell.user_ns = {} + connected_magics.shell = mock_shell + + connected_magics.sql("my_var", cell="SELECT 1 as value") + assert "my_var" in mock_shell.user_ns + assert mock_shell.user_ns["my_var"] is not None + + def test_sql_magic_empty_line_returns_none(self, connected_magics): + """Test %sql with empty line returns None.""" + result = connected_magics.sql("") + assert result is None + + def test_schema_missing_table_name_returns_usage(self, connected_magics): + """Test _schema with no table name returns usage string.""" + result = connected_magics._schema("") + assert result is not None + assert "Usage:" in result + + def test_register_parquet_dispatches(self, connected_magics, tmp_path): + """Test %register parquet calls register_parquet on context.""" + with patch.object(connected_magics._ctx, "register_parquet") as mock_reg: + connected_magics.register(f"parquet my_table {tmp_path}/file.parquet") + mock_reg.assert_called_once() + + def test_register_csv_dispatches(self, connected_magics, tmp_path): + """Test %register csv calls register_csv on context.""" + with patch.object(connected_magics._ctx, "register_csv") as mock_reg: + connected_magics.register(f"csv my_table {tmp_path}/file.csv") + mock_reg.assert_called_once() + + def test_register_unsupported_format_raises(self, connected_magics): + """Test %register with unsupported format raises NotImplementedError.""" + with pytest.raises(NotImplementedError): + connected_magics.register("xlsx my_table file.xlsx") + + def test_register_missing_table_name_returns_message(self, connected_magics): + """Test %register with missing table name returns a message.""" + result = connected_magics.register("parquet") + assert result is not None + + def test_register_missing_file_path_returns_message(self, connected_magics): + """Test %register with missing file path returns a message.""" + result = connected_magics.register("parquet my_table") + assert result is not None + + +class TestIPythonExtension: + """Tests for IPython extension loading.""" + + def test_load_extension_function_exists(self): + """Test that load_ipython_extension is defined and callable.""" + assert callable(load_ipython_extension) + + def test_load_extension_registers_magics(self): + """Test that load_ipython_extension registers BallistaMagics.""" + mock_ipython = MagicMock() + load_ipython_extension(mock_ipython) + mock_ipython.register_magics.assert_called_once_with(BallistaMagics) + + @pytest.mark.skipif(not IPYTHON_AVAILABLE, reason="IPython not available") + def test_load_extension_with_none_shell_does_not_crash(self): + """Test loading extension with None shell doesn't crash.""" + try: + load_ipython_extension(None) + except (AttributeError, TypeError): + pass # Expected when shell is None + + +class TestBallistaConnectionError: + """Tests for the BallistaConnectionError exception.""" + + def test_tables_when_disconnected_warns(self, magics): + with pytest.warns(UserWarning, match="Not connected"): + magics._tables() + + def test_schema_when_disconnected_warns(self, magics): + with pytest.warns(UserWarning, match="Not connected"): + magics._schema("some_table") + + def test_execute_sql_when_disconnected_prints_error(self, magics, capsys): + with patch("ballista.jupyter.IPYTHON_AVAILABLE", False): + result = magics._execute_sql("SELECT 1") + assert result is None + captured = capsys.readouterr() + assert "Query failed" in captured.out diff --git a/python/python/tests/test_notebook_features.py b/python/python/tests/test_notebook_features.py new file mode 100644 index 000000000..764d1d6dc --- /dev/null +++ b/python/python/tests/test_notebook_features.py @@ -0,0 +1,233 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +"""Tests for notebook-specific features in Ballista.""" + +import pytest +import tempfile +import os +from ballista import ( + BallistaSessionContext, + setup_test_cluster, + ExecutionPlanVisualization, +) + + +@pytest.fixture +def ctx(): + """Create a connected Ballista context for testing.""" + (address, port) = setup_test_cluster() + return BallistaSessionContext(address=f"df://{address}:{port}") + + +class TestExplainVisual: + """Tests for the explain_visual method.""" + + def test_explain_visual_returns_visualization(self, ctx): + """Test that explain_visual returns an ExecutionPlanVisualization.""" + df = ctx.sql("SELECT 1 as value") + viz = df.explain_visual() + assert isinstance(viz, ExecutionPlanVisualization) + + def test_explain_visual_with_analyze(self, ctx): + """Test explain_visual with analyze=True.""" + df = ctx.sql("SELECT 1 as value") + viz = df.explain_visual(analyze=True) + assert isinstance(viz, ExecutionPlanVisualization) + assert viz.analyze is True + + def test_visualization_to_dot(self, ctx): + """Test converting visualization to DOT format.""" + df = ctx.sql("SELECT 1 as value") + viz = df.explain_visual() + dot = viz.to_dot() + assert "digraph" in dot + assert "ExecutionPlan" in dot + + def test_visualization_to_svg(self, ctx): + """Test converting visualization to SVG or HTML fallback.""" + df = ctx.sql("SELECT 1 as value") + viz = df.explain_visual() + svg = viz.to_svg() + # Should return either SVG or HTML fallback + assert svg is not None + assert len(svg) > 0 + + def test_visualization_repr_html(self, ctx): + """Test HTML representation for Jupyter.""" + df = ctx.sql("SELECT 1 as value") + viz = df.explain_visual() + html = viz._repr_html_() + assert html is not None + assert len(html) > 0 + + def test_visualization_repr(self, ctx): + """Test string representation.""" + df = ctx.sql("SELECT 1 as value") + viz = df.explain_visual() + s = repr(viz) + assert "ExecutionPlanVisualization" in s + + def test_visualization_save_dot(self, ctx): + """Test saving visualization as DOT file.""" + df = ctx.sql("SELECT 1 as value") + viz = df.explain_visual() + + with tempfile.NamedTemporaryFile(suffix=".dot", delete=False) as f: + path = f.name + + try: + viz.save(path) + with open(path) as f: + content = f.read() + assert "digraph" in content + finally: + os.unlink(path) + + def test_visualization_save_svg(self, ctx): + """Test saving visualization as SVG file.""" + df = ctx.sql("SELECT 1 as value") + viz = df.explain_visual() + + with tempfile.NamedTemporaryFile(suffix=".svg", delete=False) as f: + path = f.name + + try: + viz.save(path) + with open(path) as f: + content = f.read() + # Should have some content (either SVG or HTML fallback) + assert len(content) > 0 + finally: + os.unlink(path) + + +class TestExecutionPlanVisualization: + """Tests for ExecutionPlanVisualization class.""" + + def test_creation(self): + """Test creating a visualization from plan string.""" + plan_str = """ + LogicalPlan + Projection: a, b + Filter: a > 1 + TableScan: test + """ + viz = ExecutionPlanVisualization(plan_str) + assert viz.plan_str == plan_str + assert viz.analyze is False + + def test_creation_with_analyze(self): + """Test creating a visualization with analyze flag.""" + plan_str = "LogicalPlan\n Projection: a" + viz = ExecutionPlanVisualization(plan_str, analyze=True) + assert viz.analyze is True + + def test_dot_generation(self): + """Test DOT format generation.""" + plan_str = """ +LogicalPlan + Projection: a, b + Filter: a > 1 + TableScan: test + """ + viz = ExecutionPlanVisualization(plan_str) + dot = viz.to_dot() + + assert "digraph ExecutionPlan" in dot + assert "node" in dot + assert "->" in dot # Should have edges + + def test_svg_caching(self): + """Test that SVG is cached after first generation.""" + viz = ExecutionPlanVisualization("Test plan") + svg1 = viz.to_svg() + svg2 = viz.to_svg() + assert svg1 == svg2 # Should be same cached result + + +class TestCollectWithProgress: + """Tests for the collect_with_progress method.""" + + def test_collect_with_progress_returns_batches(self, ctx): + """Test that collect_with_progress returns batches.""" + df = ctx.sql("SELECT 1 as value") + batches = df.collect_with_progress() + assert batches is not None + assert len(batches) > 0 + + def test_collect_with_progress_callback(self, ctx): + """Test collect_with_progress with callback.""" + df = ctx.sql("SELECT 1 as value") + callback_calls = [] + + def callback(status, progress): + callback_calls.append((status, progress)) + + batches = df.collect_with_progress(callback=callback) + assert batches is not None + # Callback should have been called at least once for completion + assert len(callback_calls) > 0 + + def test_collect_with_progress_matches_collect(self, ctx): + """Test that collect_with_progress returns same data as collect.""" + df = ctx.sql("SELECT 1 as value") + batches_progress = df.collect_with_progress() + batches_normal = df.collect() + + assert len(batches_progress) == len(batches_normal) + for bp, bn in zip(batches_progress, batches_normal): + assert len(bp) == len(bn) + + +class TestBallistaSessionContextTables: + """Tests for the BallistaSessionContext.tables() method.""" + + def test_tables_empty(self, ctx): + """Test listing tables when none registered.""" + # Should return empty list or at least not crash + tables = ctx.tables() + assert isinstance(tables, set) + + def test_tables_after_register_parquet(self, ctx): + """Test listing tables after registering a Parquet file.""" + ctx.register_parquet("test_parquet", "testdata/test.parquet") + tables = ctx.tables() + # The table might be in the list depending on catalog implementation + assert isinstance(tables, set) + + def test_tables_after_register_csv(self, ctx): + """Test listing tables after registering a CSV file.""" + ctx.register_csv("test_csv", "testdata/test.csv", has_header=True) + tables = ctx.tables() + assert isinstance(tables, set) + + +class TestHTMLRendering: + """Tests for HTML rendering in notebooks.""" + + def test_dataframe_repr_html(self, ctx): + """Test that DataFrame has _repr_html_ method.""" + df = ctx.sql("SELECT 1 as value") + assert hasattr(df, "_repr_html_") + + def test_dataframe_repr_html_returns_html(self, ctx): + """Test that _repr_html_ returns valid HTML.""" + df = ctx.sql("SELECT 1 as value") + html = df._repr_html_() + # The HTML should contain table tags or similar + assert html is not None diff --git a/python/uv.lock b/python/uv.lock index f032aae84..bcc52920b 100644 --- a/python/uv.lock +++ b/python/uv.lock @@ -1,9 +1,19 @@ version = 1 revision = 3 -requires-python = ">=3.9" +requires-python = ">=3.10" resolution-markers = [ - "python_full_version >= '3.10'", - "python_full_version < '3.10'", + "python_full_version >= '3.14'", + "python_full_version >= '3.11' and python_full_version < '3.14'", + "python_full_version < '3.11'", +] + +[[package]] +name = "asttokens" +version = "3.0.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/be/a5/8e3f9b6771b0b408517c82d97aed8f2036509bc247d46114925e32fe33f0/asttokens-3.0.1.tar.gz", hash = "sha256:71a4ee5de0bde6a31d64f6b13f2293ac190344478f081c3d1bccfcf5eacb0cb7", size = 62308, upload-time = "2025-11-15T16:43:48.578Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/d2/39/e7eaf1799466a4aef85b6a4fe7bd175ad2b1c6345066aa33f1f58d4b18d0/asttokens-3.0.1-py3-none-any.whl", hash = "sha256:15a3ebc0f43c2d0a50eeafea25e19046c68398e487b9f1f5b517f7c0f40f976a", size = 27047, upload-time = "2025-11-15T16:43:16.109Z" }, ] [[package]] @@ -20,29 +30,34 @@ name = "ballista" source = { editable = "." } dependencies = [ { name = "datafusion" }, - { name = "pyarrow", version = "21.0.0", source = { registry = "https://pypi.org/simple" }, marker = "python_full_version < '3.10'" }, - { name = "pyarrow", version = "23.0.0", source = { registry = "https://pypi.org/simple" }, marker = "python_full_version >= '3.10'" }, + { name = "pyarrow" }, { name = "typing-extensions", marker = "python_full_version < '3.13'" }, ] +[package.optional-dependencies] +jupyter = [ + { name = "ipython", version = "8.38.0", source = { registry = "https://pypi.org/simple" }, marker = "python_full_version < '3.11'" }, + { name = "ipython", version = "9.9.0", source = { registry = "https://pypi.org/simple" }, marker = "python_full_version >= '3.11'" }, +] + [package.dev-dependencies] dev = [ { name = "maturin" }, { name = "pygithub" }, - { name = "pytest", version = "8.4.2", source = { registry = "https://pypi.org/simple" }, marker = "python_full_version < '3.10'" }, - { name = "pytest", version = "9.0.2", source = { registry = "https://pypi.org/simple" }, marker = "python_full_version >= '3.10'" }, - { name = "pytest-asyncio", version = "1.2.0", source = { registry = "https://pypi.org/simple" }, marker = "python_full_version < '3.10'" }, - { name = "pytest-asyncio", version = "1.3.0", source = { registry = "https://pypi.org/simple" }, marker = "python_full_version >= '3.10'" }, + { name = "pytest" }, + { name = "pytest-asyncio" }, { name = "ruff" }, { name = "toml" }, ] [package.metadata] requires-dist = [ - { name = "datafusion", specifier = "==49" }, + { name = "datafusion", specifier = "==51" }, + { name = "ipython", marker = "extra == 'jupyter'", specifier = ">=8.0.0" }, { name = "pyarrow", specifier = ">=21.0.0" }, { name = "typing-extensions", marker = "python_full_version < '3.13'" }, ] +provides-extras = ["jupyter"] [package.metadata.requires-dev] dev = [ @@ -144,18 +159,6 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/a0/1d/ec1a60bd1a10daa292d3cd6bb0b359a81607154fb8165f3ec95fe003b85c/cffi-2.0.0-cp314-cp314t-win32.whl", hash = "sha256:1fc9ea04857caf665289b7a75923f2c6ed559b8298a1b8c49e59f7dd95c8481e", size = 180487, upload-time = "2025-09-08T23:23:40.423Z" }, { url = "https://files.pythonhosted.org/packages/bf/41/4c1168c74fac325c0c8156f04b6749c8b6a8f405bbf91413ba088359f60d/cffi-2.0.0-cp314-cp314t-win_amd64.whl", hash = "sha256:d68b6cef7827e8641e8ef16f4494edda8b36104d79773a334beaa1e3521430f6", size = 191726, upload-time = "2025-09-08T23:23:41.742Z" }, { url = "https://files.pythonhosted.org/packages/ae/3a/dbeec9d1ee0844c679f6bb5d6ad4e9f198b1224f4e7a32825f47f6192b0c/cffi-2.0.0-cp314-cp314t-win_arm64.whl", hash = "sha256:0a1527a803f0a659de1af2e1fd700213caba79377e27e4693648c2923da066f9", size = 184195, upload-time = "2025-09-08T23:23:43.004Z" }, - { url = "https://files.pythonhosted.org/packages/c0/cc/08ed5a43f2996a16b462f64a7055c6e962803534924b9b2f1371d8c00b7b/cffi-2.0.0-cp39-cp39-macosx_10_13_x86_64.whl", hash = "sha256:fe562eb1a64e67dd297ccc4f5addea2501664954f2692b69a76449ec7913ecbf", size = 184288, upload-time = "2025-09-08T23:23:48.404Z" }, - { url = "https://files.pythonhosted.org/packages/3d/de/38d9726324e127f727b4ecc376bc85e505bfe61ef130eaf3f290c6847dd4/cffi-2.0.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:de8dad4425a6ca6e4e5e297b27b5c824ecc7581910bf9aee86cb6835e6812aa7", size = 180509, upload-time = "2025-09-08T23:23:49.73Z" }, - { url = "https://files.pythonhosted.org/packages/9b/13/c92e36358fbcc39cf0962e83223c9522154ee8630e1df7c0b3a39a8124e2/cffi-2.0.0-cp39-cp39-manylinux1_i686.manylinux2014_i686.manylinux_2_17_i686.manylinux_2_5_i686.whl", hash = "sha256:4647afc2f90d1ddd33441e5b0e85b16b12ddec4fca55f0d9671fef036ecca27c", size = 208813, upload-time = "2025-09-08T23:23:51.263Z" }, - { url = "https://files.pythonhosted.org/packages/15/12/a7a79bd0df4c3bff744b2d7e52cc1b68d5e7e427b384252c42366dc1ecbc/cffi-2.0.0-cp39-cp39-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:3f4d46d8b35698056ec29bca21546e1551a205058ae1a181d871e278b0b28165", size = 216498, upload-time = "2025-09-08T23:23:52.494Z" }, - { url = "https://files.pythonhosted.org/packages/a3/ad/5c51c1c7600bdd7ed9a24a203ec255dccdd0ebf4527f7b922a0bde2fb6ed/cffi-2.0.0-cp39-cp39-manylinux2014_ppc64le.manylinux_2_17_ppc64le.whl", hash = "sha256:e6e73b9e02893c764e7e8d5bb5ce277f1a009cd5243f8228f75f842bf937c534", size = 203243, upload-time = "2025-09-08T23:23:53.836Z" }, - { url = "https://files.pythonhosted.org/packages/32/f2/81b63e288295928739d715d00952c8c6034cb6c6a516b17d37e0c8be5600/cffi-2.0.0-cp39-cp39-manylinux2014_s390x.manylinux_2_17_s390x.whl", hash = "sha256:cb527a79772e5ef98fb1d700678fe031e353e765d1ca2d409c92263c6d43e09f", size = 203158, upload-time = "2025-09-08T23:23:55.169Z" }, - { url = "https://files.pythonhosted.org/packages/1f/74/cc4096ce66f5939042ae094e2e96f53426a979864aa1f96a621ad128be27/cffi-2.0.0-cp39-cp39-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:61d028e90346df14fedc3d1e5441df818d095f3b87d286825dfcbd6459b7ef63", size = 216548, upload-time = "2025-09-08T23:23:56.506Z" }, - { url = "https://files.pythonhosted.org/packages/e8/be/f6424d1dc46b1091ffcc8964fa7c0ab0cd36839dd2761b49c90481a6ba1b/cffi-2.0.0-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:0f6084a0ea23d05d20c3edcda20c3d006f9b6f3fefeac38f59262e10cef47ee2", size = 218897, upload-time = "2025-09-08T23:23:57.825Z" }, - { url = "https://files.pythonhosted.org/packages/f7/e0/dda537c2309817edf60109e39265f24f24aa7f050767e22c98c53fe7f48b/cffi-2.0.0-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:1cd13c99ce269b3ed80b417dcd591415d3372bcac067009b6e0f59c7d4015e65", size = 211249, upload-time = "2025-09-08T23:23:59.139Z" }, - { url = "https://files.pythonhosted.org/packages/2b/e7/7c769804eb75e4c4b35e658dba01de1640a351a9653c3d49ca89d16ccc91/cffi-2.0.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:89472c9762729b5ae1ad974b777416bfda4ac5642423fa93bd57a09204712322", size = 218041, upload-time = "2025-09-08T23:24:00.496Z" }, - { url = "https://files.pythonhosted.org/packages/aa/d9/6218d78f920dcd7507fc16a766b5ef8f3b913cc7aa938e7fc80b9978d089/cffi-2.0.0-cp39-cp39-win32.whl", hash = "sha256:2081580ebb843f759b9f617314a24ed5738c51d2aee65d31e02f6f7a2b97707a", size = 172138, upload-time = "2025-09-08T23:24:01.7Z" }, - { url = "https://files.pythonhosted.org/packages/54/8f/a1e836f82d8e32a97e6b29cc8f641779181ac7363734f12df27db803ebda/cffi-2.0.0-cp39-cp39-win_amd64.whl", hash = "sha256:b882b3df248017dba09d6b16defe9b5c407fe32fc7c65a9c69798e6175601be9", size = 182794, upload-time = "2025-09-08T23:24:02.943Z" }, ] [[package]] @@ -244,22 +247,6 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/b0/6f/8f7af07237c34a1defe7defc565a9bc1807762f672c0fde711a4b22bf9c0/charset_normalizer-3.4.4-cp314-cp314-win32.whl", hash = "sha256:f9d332f8c2a2fcbffe1378594431458ddbef721c1769d78e2cbc06280d8155f9", size = 99940, upload-time = "2025-10-14T04:41:49.946Z" }, { url = "https://files.pythonhosted.org/packages/4b/51/8ade005e5ca5b0d80fb4aff72a3775b325bdc3d27408c8113811a7cbe640/charset_normalizer-3.4.4-cp314-cp314-win_amd64.whl", hash = "sha256:8a6562c3700cce886c5be75ade4a5db4214fda19fede41d9792d100288d8f94c", size = 107104, upload-time = "2025-10-14T04:41:51.051Z" }, { url = "https://files.pythonhosted.org/packages/da/5f/6b8f83a55bb8278772c5ae54a577f3099025f9ade59d0136ac24a0df4bde/charset_normalizer-3.4.4-cp314-cp314-win_arm64.whl", hash = "sha256:de00632ca48df9daf77a2c65a484531649261ec9f25489917f09e455cb09ddb2", size = 100743, upload-time = "2025-10-14T04:41:52.122Z" }, - { url = "https://files.pythonhosted.org/packages/46/7c/0c4760bccf082737ca7ab84a4c2034fcc06b1f21cf3032ea98bd6feb1725/charset_normalizer-3.4.4-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:a9768c477b9d7bd54bc0c86dbaebdec6f03306675526c9927c0e8a04e8f94af9", size = 209609, upload-time = "2025-10-14T04:42:10.922Z" }, - { url = "https://files.pythonhosted.org/packages/bb/a4/69719daef2f3d7f1819de60c9a6be981b8eeead7542d5ec4440f3c80e111/charset_normalizer-3.4.4-cp39-cp39-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:1bee1e43c28aa63cb16e5c14e582580546b08e535299b8b6158a7c9c768a1f3d", size = 149029, upload-time = "2025-10-14T04:42:12.38Z" }, - { url = "https://files.pythonhosted.org/packages/e6/21/8d4e1d6c1e6070d3672908b8e4533a71b5b53e71d16828cc24d0efec564c/charset_normalizer-3.4.4-cp39-cp39-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:fd44c878ea55ba351104cb93cc85e74916eb8fa440ca7903e57575e97394f608", size = 144580, upload-time = "2025-10-14T04:42:13.549Z" }, - { url = "https://files.pythonhosted.org/packages/a7/0a/a616d001b3f25647a9068e0b9199f697ce507ec898cacb06a0d5a1617c99/charset_normalizer-3.4.4-cp39-cp39-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:0f04b14ffe5fdc8c4933862d8306109a2c51e0704acfa35d51598eb45a1e89fc", size = 162340, upload-time = "2025-10-14T04:42:14.892Z" }, - { url = "https://files.pythonhosted.org/packages/85/93/060b52deb249a5450460e0585c88a904a83aec474ab8e7aba787f45e79f2/charset_normalizer-3.4.4-cp39-cp39-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:cd09d08005f958f370f539f186d10aec3377d55b9eeb0d796025d4886119d76e", size = 159619, upload-time = "2025-10-14T04:42:16.676Z" }, - { url = "https://files.pythonhosted.org/packages/dd/21/0274deb1cc0632cd587a9a0ec6b4674d9108e461cb4cd40d457adaeb0564/charset_normalizer-3.4.4-cp39-cp39-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:4fe7859a4e3e8457458e2ff592f15ccb02f3da787fcd31e0183879c3ad4692a1", size = 153980, upload-time = "2025-10-14T04:42:17.917Z" }, - { url = "https://files.pythonhosted.org/packages/28/2b/e3d7d982858dccc11b31906976323d790dded2017a0572f093ff982d692f/charset_normalizer-3.4.4-cp39-cp39-manylinux_2_31_riscv64.manylinux_2_39_riscv64.whl", hash = "sha256:fa09f53c465e532f4d3db095e0c55b615f010ad81803d383195b6b5ca6cbf5f3", size = 152174, upload-time = "2025-10-14T04:42:19.018Z" }, - { url = "https://files.pythonhosted.org/packages/6e/ff/4a269f8e35f1e58b2df52c131a1fa019acb7ef3f8697b7d464b07e9b492d/charset_normalizer-3.4.4-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:7fa17817dc5625de8a027cb8b26d9fefa3ea28c8253929b8d6649e705d2835b6", size = 151666, upload-time = "2025-10-14T04:42:20.171Z" }, - { url = "https://files.pythonhosted.org/packages/da/c9/ec39870f0b330d58486001dd8e532c6b9a905f5765f58a6f8204926b4a93/charset_normalizer-3.4.4-cp39-cp39-musllinux_1_2_armv7l.whl", hash = "sha256:5947809c8a2417be3267efc979c47d76a079758166f7d43ef5ae8e9f92751f88", size = 145550, upload-time = "2025-10-14T04:42:21.324Z" }, - { url = "https://files.pythonhosted.org/packages/75/8f/d186ab99e40e0ed9f82f033d6e49001701c81244d01905dd4a6924191a30/charset_normalizer-3.4.4-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:4902828217069c3c5c71094537a8e623f5d097858ac6ca8252f7b4d10b7560f1", size = 163721, upload-time = "2025-10-14T04:42:22.46Z" }, - { url = "https://files.pythonhosted.org/packages/96/b1/6047663b9744df26a7e479ac1e77af7134b1fcf9026243bb48ee2d18810f/charset_normalizer-3.4.4-cp39-cp39-musllinux_1_2_riscv64.whl", hash = "sha256:7c308f7e26e4363d79df40ca5b2be1c6ba9f02bdbccfed5abddb7859a6ce72cf", size = 152127, upload-time = "2025-10-14T04:42:23.712Z" }, - { url = "https://files.pythonhosted.org/packages/59/78/e5a6eac9179f24f704d1be67d08704c3c6ab9f00963963524be27c18ed87/charset_normalizer-3.4.4-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:2c9d3c380143a1fedbff95a312aa798578371eb29da42106a29019368a475318", size = 161175, upload-time = "2025-10-14T04:42:24.87Z" }, - { url = "https://files.pythonhosted.org/packages/e5/43/0e626e42d54dd2f8dd6fc5e1c5ff00f05fbca17cb699bedead2cae69c62f/charset_normalizer-3.4.4-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:cb01158d8b88ee68f15949894ccc6712278243d95f344770fa7593fa2d94410c", size = 155375, upload-time = "2025-10-14T04:42:27.246Z" }, - { url = "https://files.pythonhosted.org/packages/e9/91/d9615bf2e06f35e4997616ff31248c3657ed649c5ab9d35ea12fce54e380/charset_normalizer-3.4.4-cp39-cp39-win32.whl", hash = "sha256:2677acec1a2f8ef614c6888b5b4ae4060cc184174a938ed4e8ef690e15d3e505", size = 99692, upload-time = "2025-10-14T04:42:28.425Z" }, - { url = "https://files.pythonhosted.org/packages/d1/a9/6c040053909d9d1ef4fcab45fddec083aedc9052c10078339b47c8573ea8/charset_normalizer-3.4.4-cp39-cp39-win_amd64.whl", hash = "sha256:f8e160feb2aed042cd657a72acc0b481212ed28b1b9a95c0cee1621b524e1966", size = 107192, upload-time = "2025-10-14T04:42:29.482Z" }, - { url = "https://files.pythonhosted.org/packages/f0/c6/4fa536b2c0cd3edfb7ccf8469fa0f363ea67b7213a842b90909ca33dd851/charset_normalizer-3.4.4-cp39-cp39-win_arm64.whl", hash = "sha256:b5d84d37db046c5ca74ee7bb47dd6cbc13f80665fdde3e8040bdd3fb015ecb50", size = 100220, upload-time = "2025-10-14T04:42:30.632Z" }, { url = "https://files.pythonhosted.org/packages/0a/4c/925909008ed5a988ccbb72dcc897407e5d6d3bd72410d69e051fc0c14647/charset_normalizer-3.4.4-py3-none-any.whl", hash = "sha256:7a32c560861a02ff789ad905a2fe94e3f840803362c84fecf1851cb4cf3dc37f", size = 53402, upload-time = "2025-10-14T04:42:31.76Z" }, ] @@ -339,20 +326,28 @@ wheels = [ [[package]] name = "datafusion" -version = "49.0.0" +version = "51.0.0" source = { registry = "https://pypi.org/simple" } dependencies = [ - { name = "pyarrow", version = "21.0.0", source = { registry = "https://pypi.org/simple" }, marker = "python_full_version < '3.10'" }, - { name = "pyarrow", version = "23.0.0", source = { registry = "https://pypi.org/simple" }, marker = "python_full_version >= '3.10'" }, + { name = "pyarrow" }, { name = "typing-extensions", marker = "python_full_version < '3.13'" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/1b/81/eb7f69e184eba6f8fc748f09aa6ac39a50a4aa54e66e1529769d68d7c9e4/datafusion-49.0.0.tar.gz", hash = "sha256:f3ba4f00d56a199a90df145f43667131dbba9aea9b170cc61cd3663241a74a94", size = 183558, upload-time = "2025-08-29T01:39:45.889Z" } +sdist = { url = "https://files.pythonhosted.org/packages/2c/6d/d0e2632c93bbcca0687eeda672af3f92042ecd349df7be55da86253594a9/datafusion-51.0.0.tar.gz", hash = "sha256:1887c7d5ed3ae5d9f389e62ba869864afad4006a3f7c99ef0ca4707782a7838f", size = 193751, upload-time = "2026-01-09T13:23:41.562Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/02/16/a1438058d784deea42105b869323007cf9dd8a52dbb4551a9ef23967a235/datafusion-49.0.0-cp39-abi3-macosx_10_12_x86_64.whl", hash = "sha256:bb419810c7b1032ec4a374260d289279c5911e3fcdcfb57c9513b6d256fe0330", size = 26455326, upload-time = "2025-08-29T01:39:19.759Z" }, - { url = "https://files.pythonhosted.org/packages/83/8d/d906b27b61838002b1cf0880eaa75c741409f0f6386192f2fabee24684af/datafusion-49.0.0-cp39-abi3-macosx_11_0_arm64.whl", hash = "sha256:95d82ceda75767714629c1b5572a41bc919066156163d1ca3777418ffdbf4916", size = 23623419, upload-time = "2025-08-29T01:39:26.613Z" }, - { url = "https://files.pythonhosted.org/packages/66/4e/2a113bef5e51e56f7273f193959a80e4abd48a2ff6856cfa219dbba85600/datafusion-49.0.0-cp39-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e093a8d8cc68bf48376c2f553dcec35829be3ea2c9557516fcf93d2bf4695593", size = 29392602, upload-time = "2025-08-29T01:39:32.053Z" }, - { url = "https://files.pythonhosted.org/packages/ea/80/f49809e61f28343f303059572d9fdfedcb6f08653f34c057c2018d98ab5e/datafusion-49.0.0-cp39-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:7e9286f92f9f292a57873f8b83f10796decba98f4c022e92d929511f68e2d79a", size = 27505100, upload-time = "2025-08-29T01:39:36.398Z" }, - { url = "https://files.pythonhosted.org/packages/f6/d7/54f1d73a9e8b084965ea40531988ca25e57f7f5c77560e198483b1c7bf25/datafusion-49.0.0-cp39-abi3-win_amd64.whl", hash = "sha256:2079a2ec6e73820a21d6f8d466a1410a94f8ef3171eba2cd2461790fc0437c91", size = 28352189, upload-time = "2025-08-29T01:39:43.546Z" }, + { url = "https://files.pythonhosted.org/packages/cf/a9/7717cec053a3309be3020fe3147e3f76e5bf21295fa8adf9b52dd44ea3ff/datafusion-51.0.0-cp310-abi3-macosx_10_12_x86_64.whl", hash = "sha256:0c0d265fe3ee0dcbfa7cc3c64c7cd94fc493f38418bd79debb7ec29f29b7176e", size = 30389413, upload-time = "2026-01-09T13:23:23.266Z" }, + { url = "https://files.pythonhosted.org/packages/55/45/72c9874fd3740a4cb9d55049fdbae0df512dc5433e9f1176f3cfd970f1a1/datafusion-51.0.0-cp310-abi3-macosx_11_0_arm64.whl", hash = "sha256:43e6011db86e950bf9a21ed73cc089c2346b340a41a4f1044268af6c3a357acc", size = 26982206, upload-time = "2026-01-09T13:23:27.437Z" }, + { url = "https://files.pythonhosted.org/packages/21/ac/b32ba1f25d38fc16e7623cc4bfb7bd68db61be2ef27b2d9969ea5c865765/datafusion-51.0.0-cp310-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e76803907150159aa059d5cc9291645bbaac1b6a46d07e56035118d327b741ae", size = 33246117, upload-time = "2026-01-09T13:23:30.981Z" }, + { url = "https://files.pythonhosted.org/packages/0b/4e/437121422ef010690fc3cdd7f080203e986ba00e0e3c3b577e03f5b54ca2/datafusion-51.0.0-cp310-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:9d0cfabfe1853994adc2e6e9da5f36c1eb061102e34a2f1101fa935c6991c9e1", size = 31421867, upload-time = "2026-01-09T13:23:34.436Z" }, + { url = "https://files.pythonhosted.org/packages/db/fc/58cf27fcb85b2fd2a698253ae46213b1cbda784407e205c148f4006c1429/datafusion-51.0.0-cp310-abi3-win_amd64.whl", hash = "sha256:fd5f9abfd6669062debf0658d13e4583234c89d4df95faf381927b11cea411f5", size = 32517679, upload-time = "2026-01-09T13:23:39.615Z" }, +] + +[[package]] +name = "decorator" +version = "5.2.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/43/fa/6d96a0978d19e17b68d634497769987b16c8f4cd0a7a05048bec693caa6b/decorator-5.2.1.tar.gz", hash = "sha256:65f266143752f734b0a7cc83c46f4618af75b8c5911b00ccb61d0ac9b6da0360", size = 56711, upload-time = "2025-02-24T04:41:34.073Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/4e/8c/f3147f5c4b73e7550fe5f9352eaa956ae838d5c51eb58e7a25b9f3e2643b/decorator-5.2.1-py3-none-any.whl", hash = "sha256:d316bb415a2d9e2d2b3abcc4084c6502fc09240e292cd76a76afc106a1c8e04a", size = 9190, upload-time = "2025-02-24T04:41:32.565Z" }, ] [[package]] @@ -372,13 +367,22 @@ name = "exceptiongroup" version = "1.3.1" source = { registry = "https://pypi.org/simple" } dependencies = [ - { name = "typing-extensions", marker = "python_full_version < '3.13'" }, + { name = "typing-extensions", marker = "python_full_version < '3.11'" }, ] sdist = { url = "https://files.pythonhosted.org/packages/50/79/66800aadf48771f6b62f7eb014e352e5d06856655206165d775e675a02c9/exceptiongroup-1.3.1.tar.gz", hash = "sha256:8b412432c6055b0b7d14c310000ae93352ed6754f70fa8f7c34141f91c4e3219", size = 30371, upload-time = "2025-11-21T23:01:54.787Z" } wheels = [ { url = "https://files.pythonhosted.org/packages/8a/0e/97c33bf5009bdbac74fd2beace167cab3f978feb69cc36f1ef79360d6c4e/exceptiongroup-1.3.1-py3-none-any.whl", hash = "sha256:a7a39a3bd276781e98394987d3a5701d0c4edffb633bb7a5144577f82c773598", size = 16740, upload-time = "2025-11-21T23:01:53.443Z" }, ] +[[package]] +name = "executing" +version = "2.2.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/cc/28/c14e053b6762b1044f34a13aab6859bbf40456d37d23aa286ac24cfd9a5d/executing-2.2.1.tar.gz", hash = "sha256:3632cc370565f6648cc328b32435bd120a1e4ebb20c77e3fdde9a13cd1e533c4", size = 1129488, upload-time = "2025-09-01T09:48:10.866Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/c1/ea/53f2148663b321f21b5a606bd5f191517cf40b7072c0497d3c92c4a13b1e/executing-2.2.1-py2.py3-none-any.whl", hash = "sha256:760643d3452b4d777d295bb167ccc74c64a81df23fb5e08eff250c425a4b2017", size = 28317, upload-time = "2025-09-01T09:48:08.5Z" }, +] + [[package]] name = "idna" version = "3.11" @@ -390,26 +394,98 @@ wheels = [ [[package]] name = "iniconfig" -version = "2.1.0" +version = "2.3.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/72/34/14ca021ce8e5dfedc35312d08ba8bf51fdd999c576889fc2c24cb97f4f10/iniconfig-2.3.0.tar.gz", hash = "sha256:c76315c77db068650d49c5b56314774a7804df16fee4402c1f19d6d15d8c4730", size = 20503, upload-time = "2025-10-18T21:55:43.219Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/cb/b1/3846dd7f199d53cb17f49cba7e651e9ce294d8497c8c150530ed11865bb8/iniconfig-2.3.0-py3-none-any.whl", hash = "sha256:f631c04d2c48c52b84d0d0549c99ff3859c98df65b3101406327ecc7d53fbf12", size = 7484, upload-time = "2025-10-18T21:55:41.639Z" }, +] + +[[package]] +name = "ipython" +version = "8.38.0" source = { registry = "https://pypi.org/simple" } resolution-markers = [ - "python_full_version < '3.10'", + "python_full_version < '3.11'", +] +dependencies = [ + { name = "colorama", marker = "python_full_version < '3.11' and sys_platform == 'win32'" }, + { name = "decorator", marker = "python_full_version < '3.11'" }, + { name = "exceptiongroup", marker = "python_full_version < '3.11'" }, + { name = "jedi", marker = "python_full_version < '3.11'" }, + { name = "matplotlib-inline", marker = "python_full_version < '3.11'" }, + { name = "pexpect", marker = "python_full_version < '3.11' and sys_platform != 'emscripten' and sys_platform != 'win32'" }, + { name = "prompt-toolkit", marker = "python_full_version < '3.11'" }, + { name = "pygments", marker = "python_full_version < '3.11'" }, + { name = "stack-data", marker = "python_full_version < '3.11'" }, + { name = "traitlets", marker = "python_full_version < '3.11'" }, + { name = "typing-extensions", marker = "python_full_version < '3.11'" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/f2/97/ebf4da567aa6827c909642694d71c9fcf53e5b504f2d96afea02718862f3/iniconfig-2.1.0.tar.gz", hash = "sha256:3abbd2e30b36733fee78f9c7f7308f2d0050e88f0087fd25c2645f63c773e1c7", size = 4793, upload-time = "2025-03-19T20:09:59.721Z" } +sdist = { url = "https://files.pythonhosted.org/packages/e5/61/1810830e8b93c72dcd3c0f150c80a00c3deb229562d9423807ec92c3a539/ipython-8.38.0.tar.gz", hash = "sha256:9cfea8c903ce0867cc2f23199ed8545eb741f3a69420bfcf3743ad1cec856d39", size = 5513996, upload-time = "2026-01-05T10:59:06.901Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/2c/e1/e6716421ea10d38022b952c159d5161ca1193197fb744506875fbb87ea7b/iniconfig-2.1.0-py3-none-any.whl", hash = "sha256:9deba5723312380e77435581c6bf4935c94cbfab9b1ed33ef8d238ea168eb760", size = 6050, upload-time = "2025-03-19T20:10:01.071Z" }, + { url = "https://files.pythonhosted.org/packages/9f/df/db59624f4c71b39717c423409950ac3f2c8b2ce4b0aac843112c7fb3f721/ipython-8.38.0-py3-none-any.whl", hash = "sha256:750162629d800ac65bb3b543a14e7a74b0e88063eac9b92124d4b2aa3f6d8e86", size = 831813, upload-time = "2026-01-05T10:59:04.239Z" }, ] [[package]] -name = "iniconfig" -version = "2.3.0" +name = "ipython" +version = "9.9.0" source = { registry = "https://pypi.org/simple" } resolution-markers = [ - "python_full_version >= '3.10'", + "python_full_version >= '3.14'", + "python_full_version >= '3.11' and python_full_version < '3.14'", ] -sdist = { url = "https://files.pythonhosted.org/packages/72/34/14ca021ce8e5dfedc35312d08ba8bf51fdd999c576889fc2c24cb97f4f10/iniconfig-2.3.0.tar.gz", hash = "sha256:c76315c77db068650d49c5b56314774a7804df16fee4402c1f19d6d15d8c4730", size = 20503, upload-time = "2025-10-18T21:55:43.219Z" } +dependencies = [ + { name = "colorama", marker = "python_full_version >= '3.11' and sys_platform == 'win32'" }, + { name = "decorator", marker = "python_full_version >= '3.11'" }, + { name = "ipython-pygments-lexers", marker = "python_full_version >= '3.11'" }, + { name = "jedi", marker = "python_full_version >= '3.11'" }, + { name = "matplotlib-inline", marker = "python_full_version >= '3.11'" }, + { name = "pexpect", marker = "python_full_version >= '3.11' and sys_platform != 'emscripten' and sys_platform != 'win32'" }, + { name = "prompt-toolkit", marker = "python_full_version >= '3.11'" }, + { name = "pygments", marker = "python_full_version >= '3.11'" }, + { name = "stack-data", marker = "python_full_version >= '3.11'" }, + { name = "traitlets", marker = "python_full_version >= '3.11'" }, + { name = "typing-extensions", marker = "python_full_version == '3.11.*'" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/46/dd/fb08d22ec0c27e73c8bc8f71810709870d51cadaf27b7ddd3f011236c100/ipython-9.9.0.tar.gz", hash = "sha256:48fbed1b2de5e2c7177eefa144aba7fcb82dac514f09b57e2ac9da34ddb54220", size = 4425043, upload-time = "2026-01-05T12:36:46.233Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/cb/b1/3846dd7f199d53cb17f49cba7e651e9ce294d8497c8c150530ed11865bb8/iniconfig-2.3.0-py3-none-any.whl", hash = "sha256:f631c04d2c48c52b84d0d0549c99ff3859c98df65b3101406327ecc7d53fbf12", size = 7484, upload-time = "2025-10-18T21:55:41.639Z" }, + { url = "https://files.pythonhosted.org/packages/86/92/162cfaee4ccf370465c5af1ce36a9eacec1becb552f2033bb3584e6f640a/ipython-9.9.0-py3-none-any.whl", hash = "sha256:b457fe9165df2b84e8ec909a97abcf2ed88f565970efba16b1f7229c283d252b", size = 621431, upload-time = "2026-01-05T12:36:44.669Z" }, +] + +[[package]] +name = "ipython-pygments-lexers" +version = "1.1.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "pygments", marker = "python_full_version >= '3.11'" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/ef/4c/5dd1d8af08107f88c7f741ead7a40854b8ac24ddf9ae850afbcf698aa552/ipython_pygments_lexers-1.1.1.tar.gz", hash = "sha256:09c0138009e56b6854f9535736f4171d855c8c08a563a0dcd8022f78355c7e81", size = 8393, upload-time = "2025-01-17T11:24:34.505Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/d9/33/1f075bf72b0b747cb3288d011319aaf64083cf2efef8354174e3ed4540e2/ipython_pygments_lexers-1.1.1-py3-none-any.whl", hash = "sha256:a9462224a505ade19a605f71f8fa63c2048833ce50abc86768a0d81d876dc81c", size = 8074, upload-time = "2025-01-17T11:24:33.271Z" }, +] + +[[package]] +name = "jedi" +version = "0.19.2" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "parso" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/72/3a/79a912fbd4d8dd6fbb02bf69afd3bb72cf0c729bb3063c6f4498603db17a/jedi-0.19.2.tar.gz", hash = "sha256:4770dc3de41bde3966b02eb84fbcf557fb33cce26ad23da12c742fb50ecb11f0", size = 1231287, upload-time = "2024-11-11T01:41:42.873Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/c0/5a/9cac0c82afec3d09ccd97c8b6502d48f165f9124db81b4bcb90b4af974ee/jedi-0.19.2-py2.py3-none-any.whl", hash = "sha256:a8ef22bde8490f57fe5c7681a3c83cb58874daf72b4784de3cce5b6ef6edb5b9", size = 1572278, upload-time = "2024-11-11T01:41:40.175Z" }, +] + +[[package]] +name = "matplotlib-inline" +version = "0.2.1" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "traitlets" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/c7/74/97e72a36efd4ae2bccb3463284300f8953f199b5ffbc04cbbb0ec78f74b1/matplotlib_inline-0.2.1.tar.gz", hash = "sha256:e1ee949c340d771fc39e241ea75683deb94762c8fa5f2927ec57c83c4dffa9fe", size = 8110, upload-time = "2025-10-23T09:00:22.126Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/af/33/ee4519fa02ed11a94aef9559552f3b17bb863f2ecfe1a35dc7f548cde231/matplotlib_inline-0.2.1-py3-none-any.whl", hash = "sha256:d56ce5156ba6085e00a9d54fead6ed29a9c47e215cd1bba2e976ef39f5710a76", size = 9516, upload-time = "2025-10-23T09:00:20.675Z" }, ] [[package]] @@ -445,6 +521,27 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/20/12/38679034af332785aac8774540895e234f4d07f7545804097de4b666afd8/packaging-25.0-py3-none-any.whl", hash = "sha256:29572ef2b1f17581046b3a2227d5c611fb25ec70ca1ba8554b24b0e69331a484", size = 66469, upload-time = "2025-04-19T11:48:57.875Z" }, ] +[[package]] +name = "parso" +version = "0.8.5" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/d4/de/53e0bcf53d13e005bd8c92e7855142494f41171b34c2536b86187474184d/parso-0.8.5.tar.gz", hash = "sha256:034d7354a9a018bdce352f48b2a8a450f05e9d6ee85db84764e9b6bd96dafe5a", size = 401205, upload-time = "2025-08-23T15:15:28.028Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/16/32/f8e3c85d1d5250232a5d3477a2a28cc291968ff175caeadaf3cc19ce0e4a/parso-0.8.5-py2.py3-none-any.whl", hash = "sha256:646204b5ee239c396d040b90f9e272e9a8017c630092bf59980beb62fd033887", size = 106668, upload-time = "2025-08-23T15:15:25.663Z" }, +] + +[[package]] +name = "pexpect" +version = "4.9.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "ptyprocess" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/42/92/cc564bf6381ff43ce1f4d06852fc19a2f11d180f23dc32d9588bee2f149d/pexpect-4.9.0.tar.gz", hash = "sha256:ee7d41123f3c9911050ea2c2dac107568dc43b2d3b0c7557a33212c398ead30f", size = 166450, upload-time = "2023-11-25T09:07:26.339Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/9e/c3/059298687310d527a58bb01f3b1965787ee3b40dce76752eda8b44e9a2c5/pexpect-4.9.0-py2.py3-none-any.whl", hash = "sha256:7236d1e080e4936be2dc3e326cec0af72acf9212a7e1d060210e70a47e253523", size = 63772, upload-time = "2023-11-25T06:56:14.81Z" }, +] + [[package]] name = "pluggy" version = "1.6.0" @@ -455,65 +552,39 @@ wheels = [ ] [[package]] -name = "pyarrow" -version = "21.0.0" +name = "prompt-toolkit" +version = "3.0.52" source = { registry = "https://pypi.org/simple" } -resolution-markers = [ - "python_full_version < '3.10'", -] -sdist = { url = "https://files.pythonhosted.org/packages/ef/c2/ea068b8f00905c06329a3dfcd40d0fcc2b7d0f2e355bdb25b65e0a0e4cd4/pyarrow-21.0.0.tar.gz", hash = "sha256:5051f2dccf0e283ff56335760cbc8622cf52264d67e359d5569541ac11b6d5bc", size = 1133487, upload-time = "2025-07-18T00:57:31.761Z" } -wheels = [ - { url = "https://files.pythonhosted.org/packages/17/d9/110de31880016e2afc52d8580b397dbe47615defbf09ca8cf55f56c62165/pyarrow-21.0.0-cp310-cp310-macosx_12_0_arm64.whl", hash = "sha256:e563271e2c5ff4d4a4cbeb2c83d5cf0d4938b891518e676025f7268c6fe5fe26", size = 31196837, upload-time = "2025-07-18T00:54:34.755Z" }, - { url = "https://files.pythonhosted.org/packages/df/5f/c1c1997613abf24fceb087e79432d24c19bc6f7259cab57c2c8e5e545fab/pyarrow-21.0.0-cp310-cp310-macosx_12_0_x86_64.whl", hash = "sha256:fee33b0ca46f4c85443d6c450357101e47d53e6c3f008d658c27a2d020d44c79", size = 32659470, upload-time = "2025-07-18T00:54:38.329Z" }, - { url = "https://files.pythonhosted.org/packages/3e/ed/b1589a777816ee33ba123ba1e4f8f02243a844fed0deec97bde9fb21a5cf/pyarrow-21.0.0-cp310-cp310-manylinux_2_28_aarch64.whl", hash = "sha256:7be45519b830f7c24b21d630a31d48bcebfd5d4d7f9d3bdb49da9cdf6d764edb", size = 41055619, upload-time = "2025-07-18T00:54:42.172Z" }, - { url = "https://files.pythonhosted.org/packages/44/28/b6672962639e85dc0ac36f71ab3a8f5f38e01b51343d7aa372a6b56fa3f3/pyarrow-21.0.0-cp310-cp310-manylinux_2_28_x86_64.whl", hash = "sha256:26bfd95f6bff443ceae63c65dc7e048670b7e98bc892210acba7e4995d3d4b51", size = 42733488, upload-time = "2025-07-18T00:54:47.132Z" }, - { url = "https://files.pythonhosted.org/packages/f8/cc/de02c3614874b9089c94eac093f90ca5dfa6d5afe45de3ba847fd950fdf1/pyarrow-21.0.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:bd04ec08f7f8bd113c55868bd3fc442a9db67c27af098c5f814a3091e71cc61a", size = 43329159, upload-time = "2025-07-18T00:54:51.686Z" }, - { url = "https://files.pythonhosted.org/packages/a6/3e/99473332ac40278f196e105ce30b79ab8affab12f6194802f2593d6b0be2/pyarrow-21.0.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:9b0b14b49ac10654332a805aedfc0147fb3469cbf8ea951b3d040dab12372594", size = 45050567, upload-time = "2025-07-18T00:54:56.679Z" }, - { url = "https://files.pythonhosted.org/packages/7b/f5/c372ef60593d713e8bfbb7e0c743501605f0ad00719146dc075faf11172b/pyarrow-21.0.0-cp310-cp310-win_amd64.whl", hash = "sha256:9d9f8bcb4c3be7738add259738abdeddc363de1b80e3310e04067aa1ca596634", size = 26217959, upload-time = "2025-07-18T00:55:00.482Z" }, - { url = "https://files.pythonhosted.org/packages/94/dc/80564a3071a57c20b7c32575e4a0120e8a330ef487c319b122942d665960/pyarrow-21.0.0-cp311-cp311-macosx_12_0_arm64.whl", hash = "sha256:c077f48aab61738c237802836fc3844f85409a46015635198761b0d6a688f87b", size = 31243234, upload-time = "2025-07-18T00:55:03.812Z" }, - { url = "https://files.pythonhosted.org/packages/ea/cc/3b51cb2db26fe535d14f74cab4c79b191ed9a8cd4cbba45e2379b5ca2746/pyarrow-21.0.0-cp311-cp311-macosx_12_0_x86_64.whl", hash = "sha256:689f448066781856237eca8d1975b98cace19b8dd2ab6145bf49475478bcaa10", size = 32714370, upload-time = "2025-07-18T00:55:07.495Z" }, - { url = "https://files.pythonhosted.org/packages/24/11/a4431f36d5ad7d83b87146f515c063e4d07ef0b7240876ddb885e6b44f2e/pyarrow-21.0.0-cp311-cp311-manylinux_2_28_aarch64.whl", hash = "sha256:479ee41399fcddc46159a551705b89c05f11e8b8cb8e968f7fec64f62d91985e", size = 41135424, upload-time = "2025-07-18T00:55:11.461Z" }, - { url = "https://files.pythonhosted.org/packages/74/dc/035d54638fc5d2971cbf1e987ccd45f1091c83bcf747281cf6cc25e72c88/pyarrow-21.0.0-cp311-cp311-manylinux_2_28_x86_64.whl", hash = "sha256:40ebfcb54a4f11bcde86bc586cbd0272bac0d516cfa539c799c2453768477569", size = 42823810, upload-time = "2025-07-18T00:55:16.301Z" }, - { url = "https://files.pythonhosted.org/packages/2e/3b/89fced102448a9e3e0d4dded1f37fa3ce4700f02cdb8665457fcc8015f5b/pyarrow-21.0.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:8d58d8497814274d3d20214fbb24abcad2f7e351474357d552a8d53bce70c70e", size = 43391538, upload-time = "2025-07-18T00:55:23.82Z" }, - { url = "https://files.pythonhosted.org/packages/fb/bb/ea7f1bd08978d39debd3b23611c293f64a642557e8141c80635d501e6d53/pyarrow-21.0.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:585e7224f21124dd57836b1530ac8f2df2afc43c861d7bf3d58a4870c42ae36c", size = 45120056, upload-time = "2025-07-18T00:55:28.231Z" }, - { url = "https://files.pythonhosted.org/packages/6e/0b/77ea0600009842b30ceebc3337639a7380cd946061b620ac1a2f3cb541e2/pyarrow-21.0.0-cp311-cp311-win_amd64.whl", hash = "sha256:555ca6935b2cbca2c0e932bedd853e9bc523098c39636de9ad4693b5b1df86d6", size = 26220568, upload-time = "2025-07-18T00:55:32.122Z" }, - { url = "https://files.pythonhosted.org/packages/ca/d4/d4f817b21aacc30195cf6a46ba041dd1be827efa4a623cc8bf39a1c2a0c0/pyarrow-21.0.0-cp312-cp312-macosx_12_0_arm64.whl", hash = "sha256:3a302f0e0963db37e0a24a70c56cf91a4faa0bca51c23812279ca2e23481fccd", size = 31160305, upload-time = "2025-07-18T00:55:35.373Z" }, - { url = "https://files.pythonhosted.org/packages/a2/9c/dcd38ce6e4b4d9a19e1d36914cb8e2b1da4e6003dd075474c4cfcdfe0601/pyarrow-21.0.0-cp312-cp312-macosx_12_0_x86_64.whl", hash = "sha256:b6b27cf01e243871390474a211a7922bfbe3bda21e39bc9160daf0da3fe48876", size = 32684264, upload-time = "2025-07-18T00:55:39.303Z" }, - { url = "https://files.pythonhosted.org/packages/4f/74/2a2d9f8d7a59b639523454bec12dba35ae3d0a07d8ab529dc0809f74b23c/pyarrow-21.0.0-cp312-cp312-manylinux_2_28_aarch64.whl", hash = "sha256:e72a8ec6b868e258a2cd2672d91f2860ad532d590ce94cdf7d5e7ec674ccf03d", size = 41108099, upload-time = "2025-07-18T00:55:42.889Z" }, - { url = "https://files.pythonhosted.org/packages/ad/90/2660332eeb31303c13b653ea566a9918484b6e4d6b9d2d46879a33ab0622/pyarrow-21.0.0-cp312-cp312-manylinux_2_28_x86_64.whl", hash = "sha256:b7ae0bbdc8c6674259b25bef5d2a1d6af5d39d7200c819cf99e07f7dfef1c51e", size = 42829529, upload-time = "2025-07-18T00:55:47.069Z" }, - { url = "https://files.pythonhosted.org/packages/33/27/1a93a25c92717f6aa0fca06eb4700860577d016cd3ae51aad0e0488ac899/pyarrow-21.0.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:58c30a1729f82d201627c173d91bd431db88ea74dcaa3885855bc6203e433b82", size = 43367883, upload-time = "2025-07-18T00:55:53.069Z" }, - { url = "https://files.pythonhosted.org/packages/05/d9/4d09d919f35d599bc05c6950095e358c3e15148ead26292dfca1fb659b0c/pyarrow-21.0.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:072116f65604b822a7f22945a7a6e581cfa28e3454fdcc6939d4ff6090126623", size = 45133802, upload-time = "2025-07-18T00:55:57.714Z" }, - { url = "https://files.pythonhosted.org/packages/71/30/f3795b6e192c3ab881325ffe172e526499eb3780e306a15103a2764916a2/pyarrow-21.0.0-cp312-cp312-win_amd64.whl", hash = "sha256:cf56ec8b0a5c8c9d7021d6fd754e688104f9ebebf1bf4449613c9531f5346a18", size = 26203175, upload-time = "2025-07-18T00:56:01.364Z" }, - { url = "https://files.pythonhosted.org/packages/16/ca/c7eaa8e62db8fb37ce942b1ea0c6d7abfe3786ca193957afa25e71b81b66/pyarrow-21.0.0-cp313-cp313-macosx_12_0_arm64.whl", hash = "sha256:e99310a4ebd4479bcd1964dff9e14af33746300cb014aa4a3781738ac63baf4a", size = 31154306, upload-time = "2025-07-18T00:56:04.42Z" }, - { url = "https://files.pythonhosted.org/packages/ce/e8/e87d9e3b2489302b3a1aea709aaca4b781c5252fcb812a17ab6275a9a484/pyarrow-21.0.0-cp313-cp313-macosx_12_0_x86_64.whl", hash = "sha256:d2fe8e7f3ce329a71b7ddd7498b3cfac0eeb200c2789bd840234f0dc271a8efe", size = 32680622, upload-time = "2025-07-18T00:56:07.505Z" }, - { url = "https://files.pythonhosted.org/packages/84/52/79095d73a742aa0aba370c7942b1b655f598069489ab387fe47261a849e1/pyarrow-21.0.0-cp313-cp313-manylinux_2_28_aarch64.whl", hash = "sha256:f522e5709379d72fb3da7785aa489ff0bb87448a9dc5a75f45763a795a089ebd", size = 41104094, upload-time = "2025-07-18T00:56:10.994Z" }, - { url = "https://files.pythonhosted.org/packages/89/4b/7782438b551dbb0468892a276b8c789b8bbdb25ea5c5eb27faadd753e037/pyarrow-21.0.0-cp313-cp313-manylinux_2_28_x86_64.whl", hash = "sha256:69cbbdf0631396e9925e048cfa5bce4e8c3d3b41562bbd70c685a8eb53a91e61", size = 42825576, upload-time = "2025-07-18T00:56:15.569Z" }, - { url = "https://files.pythonhosted.org/packages/b3/62/0f29de6e0a1e33518dec92c65be0351d32d7ca351e51ec5f4f837a9aab91/pyarrow-21.0.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:731c7022587006b755d0bdb27626a1a3bb004bb56b11fb30d98b6c1b4718579d", size = 43368342, upload-time = "2025-07-18T00:56:19.531Z" }, - { url = "https://files.pythonhosted.org/packages/90/c7/0fa1f3f29cf75f339768cc698c8ad4ddd2481c1742e9741459911c9ac477/pyarrow-21.0.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:dc56bc708f2d8ac71bd1dcb927e458c93cec10b98eb4120206a4091db7b67b99", size = 45131218, upload-time = "2025-07-18T00:56:23.347Z" }, - { url = "https://files.pythonhosted.org/packages/01/63/581f2076465e67b23bc5a37d4a2abff8362d389d29d8105832e82c9c811c/pyarrow-21.0.0-cp313-cp313-win_amd64.whl", hash = "sha256:186aa00bca62139f75b7de8420f745f2af12941595bbbfa7ed3870ff63e25636", size = 26087551, upload-time = "2025-07-18T00:56:26.758Z" }, - { url = "https://files.pythonhosted.org/packages/c9/ab/357d0d9648bb8241ee7348e564f2479d206ebe6e1c47ac5027c2e31ecd39/pyarrow-21.0.0-cp313-cp313t-macosx_12_0_arm64.whl", hash = "sha256:a7a102574faa3f421141a64c10216e078df467ab9576684d5cd696952546e2da", size = 31290064, upload-time = "2025-07-18T00:56:30.214Z" }, - { url = "https://files.pythonhosted.org/packages/3f/8a/5685d62a990e4cac2043fc76b4661bf38d06efed55cf45a334b455bd2759/pyarrow-21.0.0-cp313-cp313t-macosx_12_0_x86_64.whl", hash = "sha256:1e005378c4a2c6db3ada3ad4c217b381f6c886f0a80d6a316fe586b90f77efd7", size = 32727837, upload-time = "2025-07-18T00:56:33.935Z" }, - { url = "https://files.pythonhosted.org/packages/fc/de/c0828ee09525c2bafefd3e736a248ebe764d07d0fd762d4f0929dbc516c9/pyarrow-21.0.0-cp313-cp313t-manylinux_2_28_aarch64.whl", hash = "sha256:65f8e85f79031449ec8706b74504a316805217b35b6099155dd7e227eef0d4b6", size = 41014158, upload-time = "2025-07-18T00:56:37.528Z" }, - { url = "https://files.pythonhosted.org/packages/6e/26/a2865c420c50b7a3748320b614f3484bfcde8347b2639b2b903b21ce6a72/pyarrow-21.0.0-cp313-cp313t-manylinux_2_28_x86_64.whl", hash = "sha256:3a81486adc665c7eb1a2bde0224cfca6ceaba344a82a971ef059678417880eb8", size = 42667885, upload-time = "2025-07-18T00:56:41.483Z" }, - { url = "https://files.pythonhosted.org/packages/0a/f9/4ee798dc902533159250fb4321267730bc0a107d8c6889e07c3add4fe3a5/pyarrow-21.0.0-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:fc0d2f88b81dcf3ccf9a6ae17f89183762c8a94a5bdcfa09e05cfe413acf0503", size = 43276625, upload-time = "2025-07-18T00:56:48.002Z" }, - { url = "https://files.pythonhosted.org/packages/5a/da/e02544d6997037a4b0d22d8e5f66bc9315c3671371a8b18c79ade1cefe14/pyarrow-21.0.0-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:6299449adf89df38537837487a4f8d3bd91ec94354fdd2a7d30bc11c48ef6e79", size = 44951890, upload-time = "2025-07-18T00:56:52.568Z" }, - { url = "https://files.pythonhosted.org/packages/e5/4e/519c1bc1876625fe6b71e9a28287c43ec2f20f73c658b9ae1d485c0c206e/pyarrow-21.0.0-cp313-cp313t-win_amd64.whl", hash = "sha256:222c39e2c70113543982c6b34f3077962b44fca38c0bd9e68bb6781534425c10", size = 26371006, upload-time = "2025-07-18T00:56:56.379Z" }, - { url = "https://files.pythonhosted.org/packages/3e/cc/ce4939f4b316457a083dc5718b3982801e8c33f921b3c98e7a93b7c7491f/pyarrow-21.0.0-cp39-cp39-macosx_12_0_arm64.whl", hash = "sha256:a7f6524e3747e35f80744537c78e7302cd41deee8baa668d56d55f77d9c464b3", size = 31211248, upload-time = "2025-07-18T00:56:59.7Z" }, - { url = "https://files.pythonhosted.org/packages/1f/c2/7a860931420d73985e2f340f06516b21740c15b28d24a0e99a900bb27d2b/pyarrow-21.0.0-cp39-cp39-macosx_12_0_x86_64.whl", hash = "sha256:203003786c9fd253ebcafa44b03c06983c9c8d06c3145e37f1b76a1f317aeae1", size = 32676896, upload-time = "2025-07-18T00:57:03.884Z" }, - { url = "https://files.pythonhosted.org/packages/68/a8/197f989b9a75e59b4ca0db6a13c56f19a0ad8a298c68da9cc28145e0bb97/pyarrow-21.0.0-cp39-cp39-manylinux_2_28_aarch64.whl", hash = "sha256:3b4d97e297741796fead24867a8dabf86c87e4584ccc03167e4a811f50fdf74d", size = 41067862, upload-time = "2025-07-18T00:57:07.587Z" }, - { url = "https://files.pythonhosted.org/packages/fa/82/6ecfa89487b35aa21accb014b64e0a6b814cc860d5e3170287bf5135c7d8/pyarrow-21.0.0-cp39-cp39-manylinux_2_28_x86_64.whl", hash = "sha256:898afce396b80fdda05e3086b4256f8677c671f7b1d27a6976fa011d3fd0a86e", size = 42747508, upload-time = "2025-07-18T00:57:13.917Z" }, - { url = "https://files.pythonhosted.org/packages/3b/b7/ba252f399bbf3addc731e8643c05532cf32e74cebb5e32f8f7409bc243cf/pyarrow-21.0.0-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:067c66ca29aaedae08218569a114e413b26e742171f526e828e1064fcdec13f4", size = 43345293, upload-time = "2025-07-18T00:57:19.828Z" }, - { url = "https://files.pythonhosted.org/packages/ff/0a/a20819795bd702b9486f536a8eeb70a6aa64046fce32071c19ec8230dbaa/pyarrow-21.0.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:0c4e75d13eb76295a49e0ea056eb18dbd87d81450bfeb8afa19a7e5a75ae2ad7", size = 45060670, upload-time = "2025-07-18T00:57:24.477Z" }, - { url = "https://files.pythonhosted.org/packages/10/15/6b30e77872012bbfe8265d42a01d5b3c17ef0ac0f2fae531ad91b6a6c02e/pyarrow-21.0.0-cp39-cp39-win_amd64.whl", hash = "sha256:cdc4c17afda4dab2a9c0b79148a43a7f4e1094916b3e18d8975bfd6d6d52241f", size = 26227521, upload-time = "2025-07-18T00:57:29.119Z" }, +dependencies = [ + { name = "wcwidth" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/a1/96/06e01a7b38dce6fe1db213e061a4602dd6032a8a97ef6c1a862537732421/prompt_toolkit-3.0.52.tar.gz", hash = "sha256:28cde192929c8e7321de85de1ddbe736f1375148b02f2e17edd840042b1be855", size = 434198, upload-time = "2025-08-27T15:24:02.057Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/84/03/0d3ce49e2505ae70cf43bc5bb3033955d2fc9f932163e84dc0779cc47f48/prompt_toolkit-3.0.52-py3-none-any.whl", hash = "sha256:9aac639a3bbd33284347de5ad8d68ecc044b91a762dc39b7c21095fcd6a19955", size = 391431, upload-time = "2025-08-27T15:23:59.498Z" }, +] + +[[package]] +name = "ptyprocess" +version = "0.7.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/20/e5/16ff212c1e452235a90aeb09066144d0c5a6a8c0834397e03f5224495c4e/ptyprocess-0.7.0.tar.gz", hash = "sha256:5c5d0a3b48ceee0b48485e0c26037c0acd7d29765ca3fbb5cb3831d347423220", size = 70762, upload-time = "2020-12-28T15:15:30.155Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/22/a6/858897256d0deac81a172289110f31629fc4cee19b6f01283303e18c8db3/ptyprocess-0.7.0-py2.py3-none-any.whl", hash = "sha256:4b41f3967fce3af57cc7e94b888626c18bf37a083e3651ca8feeb66d492fef35", size = 13993, upload-time = "2020-12-28T15:15:28.35Z" }, +] + +[[package]] +name = "pure-eval" +version = "0.2.3" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/cd/05/0a34433a064256a578f1783a10da6df098ceaa4a57bbeaa96a6c0352786b/pure_eval-0.2.3.tar.gz", hash = "sha256:5f4e983f40564c576c7c8635ae88db5956bb2229d7e9237d03b3c0b0190eaf42", size = 19752, upload-time = "2024-07-21T12:58:21.801Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/8e/37/efad0257dc6e593a18957422533ff0f87ede7c9c6ea010a2177d738fb82f/pure_eval-0.2.3-py3-none-any.whl", hash = "sha256:1db8e35b67b3d218d818ae653e27f06c3aa420901fa7b081ca98cbedc874e0d0", size = 11842, upload-time = "2024-07-21T12:58:20.04Z" }, ] [[package]] name = "pyarrow" version = "23.0.0" source = { registry = "https://pypi.org/simple" } -resolution-markers = [ - "python_full_version >= '3.10'", -] sdist = { url = "https://files.pythonhosted.org/packages/01/33/ffd9c3eb087fa41dd79c3cf20c4c0ae3cdb877c4f8e1107a446006344924/pyarrow-23.0.0.tar.gz", hash = "sha256:180e3150e7edfcd182d3d9afba72f7cf19839a497cc76555a8dce998a8f67615", size = 1167185, upload-time = "2026-01-18T16:19:42.218Z" } wheels = [ { url = "https://files.pythonhosted.org/packages/ae/2f/23e042a5aa99bcb15e794e14030e8d065e00827e846e53a66faec73c7cd6/pyarrow-23.0.0-cp310-cp310-macosx_12_0_arm64.whl", hash = "sha256:cbdc2bf5947aa4d462adcf8453cf04aee2f7932653cb67a27acd96e5e8528a67", size = 34281861, upload-time = "2026-01-18T16:13:34.332Z" }, @@ -651,76 +722,32 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/29/7d/5945b5af29534641820d3bd7b00962abbbdfee84ec7e19f0d5b3175f9a31/pynacl-1.6.2-cp38-abi3-win_arm64.whl", hash = "sha256:834a43af110f743a754448463e8fd61259cd4ab5bbedcf70f9dabad1d28a394c", size = 184801, upload-time = "2026-01-01T17:32:36.309Z" }, ] -[[package]] -name = "pytest" -version = "8.4.2" -source = { registry = "https://pypi.org/simple" } -resolution-markers = [ - "python_full_version < '3.10'", -] -dependencies = [ - { name = "colorama", marker = "python_full_version < '3.10' and sys_platform == 'win32'" }, - { name = "exceptiongroup", marker = "python_full_version < '3.10'" }, - { name = "iniconfig", version = "2.1.0", source = { registry = "https://pypi.org/simple" }, marker = "python_full_version < '3.10'" }, - { name = "packaging", marker = "python_full_version < '3.10'" }, - { name = "pluggy", marker = "python_full_version < '3.10'" }, - { name = "pygments", marker = "python_full_version < '3.10'" }, - { name = "tomli", marker = "python_full_version < '3.10'" }, -] -sdist = { url = "https://files.pythonhosted.org/packages/a3/5c/00a0e072241553e1a7496d638deababa67c5058571567b92a7eaa258397c/pytest-8.4.2.tar.gz", hash = "sha256:86c0d0b93306b961d58d62a4db4879f27fe25513d4b969df351abdddb3c30e01", size = 1519618, upload-time = "2025-09-04T14:34:22.711Z" } -wheels = [ - { url = "https://files.pythonhosted.org/packages/a8/a4/20da314d277121d6534b3a980b29035dcd51e6744bd79075a6ce8fa4eb8d/pytest-8.4.2-py3-none-any.whl", hash = "sha256:872f880de3fc3a5bdc88a11b39c9710c3497a547cfa9320bc3c5e62fbf272e79", size = 365750, upload-time = "2025-09-04T14:34:20.226Z" }, -] - [[package]] name = "pytest" version = "9.0.2" source = { registry = "https://pypi.org/simple" } -resolution-markers = [ - "python_full_version >= '3.10'", -] dependencies = [ - { name = "colorama", marker = "python_full_version >= '3.10' and sys_platform == 'win32'" }, - { name = "exceptiongroup", marker = "python_full_version == '3.10.*'" }, - { name = "iniconfig", version = "2.3.0", source = { registry = "https://pypi.org/simple" }, marker = "python_full_version >= '3.10'" }, - { name = "packaging", marker = "python_full_version >= '3.10'" }, - { name = "pluggy", marker = "python_full_version >= '3.10'" }, - { name = "pygments", marker = "python_full_version >= '3.10'" }, - { name = "tomli", marker = "python_full_version == '3.10.*'" }, + { name = "colorama", marker = "sys_platform == 'win32'" }, + { name = "exceptiongroup", marker = "python_full_version < '3.11'" }, + { name = "iniconfig" }, + { name = "packaging" }, + { name = "pluggy" }, + { name = "pygments" }, + { name = "tomli", marker = "python_full_version < '3.11'" }, ] sdist = { url = "https://files.pythonhosted.org/packages/d1/db/7ef3487e0fb0049ddb5ce41d3a49c235bf9ad299b6a25d5780a89f19230f/pytest-9.0.2.tar.gz", hash = "sha256:75186651a92bd89611d1d9fc20f0b4345fd827c41ccd5c299a868a05d70edf11", size = 1568901, upload-time = "2025-12-06T21:30:51.014Z" } wheels = [ { url = "https://files.pythonhosted.org/packages/3b/ab/b3226f0bd7cdcf710fbede2b3548584366da3b19b5021e74f5bde2a8fa3f/pytest-9.0.2-py3-none-any.whl", hash = "sha256:711ffd45bf766d5264d487b917733b453d917afd2b0ad65223959f59089f875b", size = 374801, upload-time = "2025-12-06T21:30:49.154Z" }, ] -[[package]] -name = "pytest-asyncio" -version = "1.2.0" -source = { registry = "https://pypi.org/simple" } -resolution-markers = [ - "python_full_version < '3.10'", -] -dependencies = [ - { name = "backports-asyncio-runner", marker = "python_full_version < '3.10'" }, - { name = "pytest", version = "8.4.2", source = { registry = "https://pypi.org/simple" }, marker = "python_full_version < '3.10'" }, - { name = "typing-extensions", marker = "python_full_version < '3.10'" }, -] -sdist = { url = "https://files.pythonhosted.org/packages/42/86/9e3c5f48f7b7b638b216e4b9e645f54d199d7abbbab7a64a13b4e12ba10f/pytest_asyncio-1.2.0.tar.gz", hash = "sha256:c609a64a2a8768462d0c99811ddb8bd2583c33fd33cf7f21af1c142e824ffb57", size = 50119, upload-time = "2025-09-12T07:33:53.816Z" } -wheels = [ - { url = "https://files.pythonhosted.org/packages/04/93/2fa34714b7a4ae72f2f8dad66ba17dd9a2c793220719e736dda28b7aec27/pytest_asyncio-1.2.0-py3-none-any.whl", hash = "sha256:8e17ae5e46d8e7efe51ab6494dd2010f4ca8dae51652aa3c8d55acf50bfb2e99", size = 15095, upload-time = "2025-09-12T07:33:52.639Z" }, -] - [[package]] name = "pytest-asyncio" version = "1.3.0" source = { registry = "https://pypi.org/simple" } -resolution-markers = [ - "python_full_version >= '3.10'", -] dependencies = [ - { name = "backports-asyncio-runner", marker = "python_full_version == '3.10.*'" }, - { name = "pytest", version = "9.0.2", source = { registry = "https://pypi.org/simple" }, marker = "python_full_version >= '3.10'" }, - { name = "typing-extensions", marker = "python_full_version >= '3.10' and python_full_version < '3.13'" }, + { name = "backports-asyncio-runner", marker = "python_full_version < '3.11'" }, + { name = "pytest" }, + { name = "typing-extensions", marker = "python_full_version < '3.13'" }, ] sdist = { url = "https://files.pythonhosted.org/packages/90/2c/8af215c0f776415f3590cac4f9086ccefd6fd463befeae41cd4d3f193e5a/pytest_asyncio-1.3.0.tar.gz", hash = "sha256:d7f52f36d231b80ee124cd216ffb19369aa168fc10095013c6b014a34d3ee9e5", size = 50087, upload-time = "2025-11-10T16:07:47.256Z" } wheels = [ @@ -768,6 +795,20 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/4d/e1/7348090988095e4e39560cfc2f7555b1b2a7357deba19167b600fdf5215d/ruff-0.14.13-py3-none-win_arm64.whl", hash = "sha256:7ab819e14f1ad9fe39f246cfcc435880ef7a9390d81a2b6ac7e01039083dd247", size = 13080224, upload-time = "2026-01-15T20:14:45.853Z" }, ] +[[package]] +name = "stack-data" +version = "0.6.3" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "asttokens" }, + { name = "executing" }, + { name = "pure-eval" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/28/e3/55dcc2cfbc3ca9c29519eb6884dd1415ecb53b0e934862d3559ddcb7e20b/stack_data-0.6.3.tar.gz", hash = "sha256:836a778de4fec4dcd1dcd89ed8abff8a221f58308462e1c4aa2a3cf30148f0b9", size = 44707, upload-time = "2023-09-30T13:58:05.479Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/f1/7b/ce1eafaf1a76852e2ec9b22edecf1daa58175c090266e9f6c64afcd81d91/stack_data-0.6.3-py3-none-any.whl", hash = "sha256:d5558e0c25a4cb0853cddad3d77da9891a08cb85dd9f9f91b9f8cd66e511e695", size = 24521, upload-time = "2023-09-30T13:58:03.53Z" }, +] + [[package]] name = "toml" version = "0.10.2" @@ -831,6 +872,15 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/23/d1/136eb2cb77520a31e1f64cbae9d33ec6df0d78bdf4160398e86eec8a8754/tomli-2.4.0-py3-none-any.whl", hash = "sha256:1f776e7d669ebceb01dee46484485f43a4048746235e683bcdffacdf1fb4785a", size = 14477, upload-time = "2026-01-11T11:22:37.446Z" }, ] +[[package]] +name = "traitlets" +version = "5.14.3" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/eb/79/72064e6a701c2183016abbbfedaba506d81e30e232a68c9f0d6f6fcd1574/traitlets-5.14.3.tar.gz", hash = "sha256:9ed0579d3502c94b4b3732ac120375cda96f923114522847de4b3bb98b96b6b7", size = 161621, upload-time = "2024-04-19T11:11:49.746Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/00/c0/8f5d070730d7836adc9c9b6408dec68c6ced86b304a9b26a14df072a6e8c/traitlets-5.14.3-py3-none-any.whl", hash = "sha256:b74e89e397b1ed28cc831db7aea759ba6640cb3de13090ca145426688ff1ac4f", size = 85359, upload-time = "2024-04-19T11:11:46.763Z" }, +] + [[package]] name = "typing-extensions" version = "4.15.0" @@ -849,6 +899,15 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/39/08/aaaad47bc4e9dc8c725e68f9d04865dbcb2052843ff09c97b08904852d84/urllib3-2.6.3-py3-none-any.whl", hash = "sha256:bf272323e553dfb2e87d9bfd225ca7b0f467b919d7bbd355436d3fd37cb0acd4", size = 131584, upload-time = "2026-01-07T16:24:42.685Z" }, ] +[[package]] +name = "wcwidth" +version = "0.5.3" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/c2/62/a7c072fbfefb2980a00f99ca994279cb9ecf310cb2e6b2a4d2a28fe192b3/wcwidth-0.5.3.tar.gz", hash = "sha256:53123b7af053c74e9fe2e92ac810301f6139e64379031f7124574212fb3b4091", size = 157587, upload-time = "2026-01-31T03:52:10.92Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/3c/c1/d73f12f8cdb1891334a2ccf7389eed244d3941e74d80dd220badb937f3fb/wcwidth-0.5.3-py3-none-any.whl", hash = "sha256:d584eff31cd4753e1e5ff6c12e1edfdb324c995713f75d26c29807bb84bf649e", size = 92981, upload-time = "2026-01-31T03:52:09.14Z" }, +] + [[package]] name = "wrapt" version = "2.0.1" @@ -939,17 +998,5 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/1d/f6/a1de4bd3653afdf91d250ca5c721ee51195df2b61a4603d4b373aa804d1d/wrapt-2.0.1-cp314-cp314t-win32.whl", hash = "sha256:be9e84e91d6497ba62594158d3d31ec0486c60055c49179edc51ee43d095f79c", size = 60609, upload-time = "2025-11-07T00:45:03.315Z" }, { url = "https://files.pythonhosted.org/packages/01/3a/07cd60a9d26fe73efead61c7830af975dfdba8537632d410462672e4432b/wrapt-2.0.1-cp314-cp314t-win_amd64.whl", hash = "sha256:61c4956171c7434634401db448371277d07032a81cc21c599c22953374781395", size = 64038, upload-time = "2025-11-07T00:45:00.948Z" }, { url = "https://files.pythonhosted.org/packages/41/99/8a06b8e17dddbf321325ae4eb12465804120f699cd1b8a355718300c62da/wrapt-2.0.1-cp314-cp314t-win_arm64.whl", hash = "sha256:35cdbd478607036fee40273be8ed54a451f5f23121bd9d4be515158f9498f7ad", size = 60634, upload-time = "2025-11-07T00:45:02.087Z" }, - { url = "https://files.pythonhosted.org/packages/c6/1f/5af0ae22368ec69067a577f9e07a0dd2619a1f63aabc2851263679942667/wrapt-2.0.1-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:68424221a2dc00d634b54f92441914929c5ffb1c30b3b837343978343a3512a3", size = 77478, upload-time = "2025-11-07T00:45:16.65Z" }, - { url = "https://files.pythonhosted.org/packages/8c/b7/fd6b563aada859baabc55db6aa71b8afb4a3ceb8bc33d1053e4c7b5e0109/wrapt-2.0.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:6bd1a18f5a797fe740cb3d7a0e853a8ce6461cc62023b630caec80171a6b8097", size = 60687, upload-time = "2025-11-07T00:45:17.896Z" }, - { url = "https://files.pythonhosted.org/packages/0f/8c/9ededfff478af396bcd081076986904bdca336d9664d247094150c877dcb/wrapt-2.0.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:fb3a86e703868561c5cad155a15c36c716e1ab513b7065bd2ac8ed353c503333", size = 61563, upload-time = "2025-11-07T00:45:19.109Z" }, - { url = "https://files.pythonhosted.org/packages/ab/a7/d795a1aa2b6ab20ca21157fe03cbfc6aa7e870a88ac3b4ea189e2f6c79f0/wrapt-2.0.1-cp39-cp39-manylinux1_x86_64.manylinux_2_28_x86_64.manylinux_2_5_x86_64.whl", hash = "sha256:5dc1b852337c6792aa111ca8becff5bacf576bf4a0255b0f05eb749da6a1643e", size = 113395, upload-time = "2025-11-07T00:45:21.551Z" }, - { url = "https://files.pythonhosted.org/packages/61/32/56cde2bbf95f2d5698a1850a765520aa86bc7ae0f95b8ec80b6f2e2049bb/wrapt-2.0.1-cp39-cp39-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:c046781d422f0830de6329fa4b16796096f28a92c8aef3850674442cdcb87b7f", size = 115362, upload-time = "2025-11-07T00:45:22.809Z" }, - { url = "https://files.pythonhosted.org/packages/cf/53/8d3cc433847c219212c133a3e8305bd087b386ef44442ff39189e8fa62ac/wrapt-2.0.1-cp39-cp39-manylinux_2_31_riscv64.manylinux_2_39_riscv64.whl", hash = "sha256:f73f9f7a0ebd0db139253d27e5fc8d2866ceaeef19c30ab5d69dcbe35e1a6981", size = 111766, upload-time = "2025-11-07T00:45:20.294Z" }, - { url = "https://files.pythonhosted.org/packages/b8/d3/14b50c2d0463c0dcef8f388cb1527ed7bbdf0972b9fd9976905f36c77ebf/wrapt-2.0.1-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:b667189cf8efe008f55bbda321890bef628a67ab4147ebf90d182f2dadc78790", size = 114560, upload-time = "2025-11-07T00:45:24.054Z" }, - { url = "https://files.pythonhosted.org/packages/3a/b8/4f731ff178f77ae55385586de9ff4b4261e872cf2ced4875e6c976fbcb8b/wrapt-2.0.1-cp39-cp39-musllinux_1_2_riscv64.whl", hash = "sha256:a9a83618c4f0757557c077ef71d708ddd9847ed66b7cc63416632af70d3e2308", size = 110999, upload-time = "2025-11-07T00:45:25.596Z" }, - { url = "https://files.pythonhosted.org/packages/fe/bb/5f1bb0f9ae9d12e19f1d71993d052082062603e83fe3e978377f918f054d/wrapt-2.0.1-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:1e9b121e9aeb15df416c2c960b8255a49d44b4038016ee17af03975992d03931", size = 113164, upload-time = "2025-11-07T00:45:26.8Z" }, - { url = "https://files.pythonhosted.org/packages/ad/f6/f3a3c623d3065c7bf292ee0b73566236b562d5ed894891bd8e435762b618/wrapt-2.0.1-cp39-cp39-win32.whl", hash = "sha256:1f186e26ea0a55f809f232e92cc8556a0977e00183c3ebda039a807a42be1494", size = 58028, upload-time = "2025-11-07T00:45:30.943Z" }, - { url = "https://files.pythonhosted.org/packages/24/78/647c609dfa18063a7fcd5c23f762dd006be401cc9206314d29c9b0b12078/wrapt-2.0.1-cp39-cp39-win_amd64.whl", hash = "sha256:bf4cb76f36be5de950ce13e22e7fdf462b35b04665a12b64f3ac5c1bbbcf3728", size = 60380, upload-time = "2025-11-07T00:45:28.341Z" }, - { url = "https://files.pythonhosted.org/packages/07/90/0c14b241d18d80ddf4c847a5f52071e126e8a6a9e5a8a7952add8ef0d766/wrapt-2.0.1-cp39-cp39-win_arm64.whl", hash = "sha256:d6cc985b9c8b235bd933990cdbf0f891f8e010b65a3911f7a55179cd7b0fc57b", size = 58895, upload-time = "2025-11-07T00:45:29.527Z" }, { url = "https://files.pythonhosted.org/packages/15/d1/b51471c11592ff9c012bd3e2f7334a6ff2f42a7aed2caffcf0bdddc9cb89/wrapt-2.0.1-py3-none-any.whl", hash = "sha256:4d2ce1bf1a48c5277d7969259232b57645aae5686dba1eaeade39442277afbca", size = 44046, upload-time = "2025-11-07T00:45:32.116Z" }, ]