Skip to content

Implement general purpose async functions #1

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 2 commits into from
Feb 3, 2025

Conversation

alamb
Copy link
Contributor

@alamb alamb commented Jan 9, 2025

This PR implements what I think is a general purpose framework for implementing async user defined functions.

The high level design is to handle async functions with a special new execution plan

I will comment more inline about the design.

When run with cargo run this program shows:

++
++
+-------+
| count |
+-------+
| 3     |
+-------+
+-------+
| count |
+-------+
| 3     |
+-------+
+-------+
| count |
+-------+
| 3     |
+-------+
+---------------+---------------------------------------------------------------------------------------------------------------------------------------------+
| plan_type     | plan                                                                                                                                        |
+---------------+---------------------------------------------------------------------------------------------------------------------------------------------+
| logical_plan  | Projection: llm_bool(Utf8("If all of them are Aisa countries: {}, {}, {}"), t1.c1, t1.c2, t1.c3)                                            |
|               |   TableScan: t1 projection=[c1, c2, c3]                                                                                                     |
| physical_plan | ProjectionExec: expr=[__async_fn_0@3 as llm_bool(Utf8("If all of them are Aisa countries: {}, {}, {}"),t1.c1,t1.c2,t1.c3)]                  |
|               |   AsyncFuncExec: async_expr=[async_expr(name=__async_fn_0, expr=llm_bool(If all of them are Aisa countries: {}, {}, {}, c1@0, c2@1, c3@2))] |
|               |     MemoryExec: partitions=1, partition_sizes=[3]                                                                                           |
|               |                                                                                                                                             |
+---------------+---------------------------------------------------------------------------------------------------------------------------------------------+
+-----------------------------------------------------------------------------------+
| llm_bool(Utf8("If all of them are Aisa countries: {}, {}, {}"),t1.c1,t1.c2,t1.c3) |
+-----------------------------------------------------------------------------------+
| false                                                                             |
| true                                                                              |
| false                                                                             |
| false                                                                             |
| false                                                                             |
| true                                                                              |
| true                                                                              |
| false                                                                             |
| false                                                                             |
+-----------------------------------------------------------------------------------+

@alamb

This comment was marked as outdated.

@alamb alamb force-pushed the alamb/make_calling_futures_easier branch from 5cb6e4e to 90bd4f6 Compare January 11, 2025 13:21
/// A scalar UDF that will be bypassed when planning logical plan.
/// This is used to register the remote function to the context. The function should not be
/// invoked by DataFusion. It's only used to generate the logical plan and unparsed them to SQL.
/// A scalar UDF that can invoke using async methods
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Here is the the new API. At a high level it is meant to mimic ScalarUDFImpl except that it has a async invoke function

pub struct AsyncFuncRule {}

impl PhysicalOptimizerRule for AsyncFuncRule {
/// Insert a AsyncFunctionNode node in front of this projection if there are any async functions in it
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Here is the high level design: add a new node before a ProjectionExec that does the actual async calls

.with_optimizer_rules(vec![])
.with_query_planner(Arc::new(LLMQueryPlanner {}))
.with_physical_optimizer_rules(vec![])
.with_physical_optimizer_rule(Arc::new(AsyncFuncRule {}))
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Here is how to use the new code: add the new optimizer rule:

Ok(())
}

/// This is a simple example of a UDF that takes a string, invokes a (remote) LLM function
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Now, define a struct that implements AsyncScalarUDFImpl

Ok(DataType::Boolean)
}

async fn invoke_async(&self, args: &RecordBatch) -> Result<ArrayRef> {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Here is the function that is invoked (it is async) and should be able to do any network and other calls

@alamb alamb changed the title WIP: Rework projection handling for async functions Rework projection handling for async functions Jan 11, 2025
@alamb
Copy link
Contributor Author

alamb commented Jan 11, 2025

@alamb alamb changed the title Rework projection handling for async functions Implement general purpose async functions Jan 11, 2025
fn return_type(&self, _arg_types: &[DataType]) -> Result<DataType>;

/// Invoke the function asynchronously with the async arguments
async fn invoke_async(&self, args: &RecordBatch) -> Result<ArrayRef>;

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wonder whether this should return a Stream of ArrayRef, so that internally you can batch the calls to an external system with the right batch size ? In case of LLM there might be also a problem with the context, I suppose...

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That is also an excellent question -- the current situation is that Datafusion handles the batching (aka target_size) -- so normally will pass 8k rows or whatever to the`

I think we could potentially make the API something like:

    fn invoke_async_stream(&self, input: SendableRecordBatchStream) -> Result<SendableRecordBatchStream>;

but I think that might be tricker to code / get right

In terms of LLM context, this particualr PR only adds async scalar functions. I think we could likely do something similar to with window and aggregate functions, which might more naturally map to context 🤔

let schema_captured = schema_captured.clone();

async move {
let batch = batch?;

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

minor, would moving this invocation of the ? operator save a task in case of an error?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes, you are right -- that would be an improvement 👍

Copy link
Owner

@goldmedal goldmedal left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks, @alamb!
That's a reasonable way to implement an async function in DataFusion.
It's beneficial! I'll follow this approach and move on to the next step.

Sorry for the late reply—I just got back to work. 🙇

@goldmedal goldmedal merged commit 3440f25 into goldmedal:master Feb 3, 2025
@alamb alamb deleted the alamb/make_calling_futures_easier branch February 4, 2025 16:53
@alamb
Copy link
Contributor Author

alamb commented Feb 4, 2025

Sorry for the late reply—I just got back to work. 🙇

Welcome back!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants