You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@arrow.apache.org by "Ben Sully (Jira)" <ji...@apache.org> on 2020/12/30 10:00:00 UTC
[jira] [Commented] (ARROW-9480) [Rust] [DataFusion] All DataFusion
execution plan traits should require Send + Sync
[ https://issues.apache.org/jira/browse/ARROW-9480?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17256415#comment-17256415 ]
Ben Sully commented on ARROW-9480:
----------------------------------
I'm running into this issue when experimenting with running a DataFusion engine behind an (async) API. Here's a minimal repro (requires tokio 0.2 and datafusion as deps):
{code:java}
use datafusion::execution::context::ExecutionContext;
async fn query(ctx: &mut ExecutionContext) {
let df = ctx.sql("SELECT * FROM foo LIMIT 10").unwrap();
df.collect().await.unwrap();
}
#[tokio::main]
async fn main() {
let mut ctx = ExecutionContext::new();
tokio::task::spawn_blocking(|| query(&mut ctx)).await;
}
{code}
The compiler complains because the future created to represent `query` is not `Send` or `Sync` because the value created by `ctx.sql` is a `dyn DataFrame`, which is not currently `Send` or `Sync`. This means that even though I want to spawn the task on the blocking thread pool, I can't.
I have a similar issue when trying to run an async function which uses a `dyn TableProvider` since that isn't Send/Sync either.
Here's the compiler error:
{code:java}
❮ cargo c <<<
Checking datafusion-repro v0.1.0 (/tmp/datafusion-repro)
error: future cannot be sent between threads safely
--> src/main.rs:11:5
|
11 | tokio::task::spawn_blocking(|| query(&mut ctx)).await;
| ^^^^^^^^^^^^^^^^^^^^^^^^^^^ future returned by `query` is not `Send`
|
::: /home/ben/.cargo/registry/src/github.com-1ecc6299db9ec823/tokio-0.2.24/src/task/blocking.rs:115:12
|
115 | R: Send + 'static,
| ---- required by this bound in `spawn_blocking`
|
= help: the trait `Send` is not implemented for `dyn DataFrame`
note: future is not `Send` as this value is used across an await
--> src/main.rs:5:5
|
4 | let df = ctx.sql("SELECT * FROM foo LIMIT 10").unwrap();
| -- has type `Arc<dyn DataFrame>` which is not `Send`
5 | df.collect().await.unwrap();
| ^^^^^^^^^^^^^^^^^^ await occurs here, with `df` maybe used later
6 | }
| - `df` is later dropped here
error: future cannot be sent between threads safely
--> src/main.rs:11:5
|
11 | tokio::task::spawn_blocking(|| query(&mut ctx)).await;
| ^^^^^^^^^^^^^^^^^^^^^^^^^^^ future returned by `query` is not `Send`
|
::: /home/ben/.cargo/registry/src/github.com-1ecc6299db9ec823/tokio-0.2.24/src/task/blocking.rs:115:12
|
115 | R: Send + 'static,
| ---- required by this bound in `spawn_blocking`
|
= help: the trait `Sync` is not implemented for `dyn DataFrame`
note: future is not `Send` as this value is used across an await
--> src/main.rs:5:5
|
4 | let df = ctx.sql("SELECT * FROM foo LIMIT 10").unwrap();
| -- has type `Arc<dyn DataFrame>` which is not `Send`
5 | df.collect().await.unwrap();
| ^^^^^^^^^^^^^^^^^^ await occurs here, with `df` maybe used later
6 | }
| - `df` is later dropped here
error: aborting due to 2 previous errors
error: could not compile `datafusion-repro`
{code}
> [Rust] [DataFusion] All DataFusion execution plan traits should require Send + Sync
> -----------------------------------------------------------------------------------
>
> Key: ARROW-9480
> URL: https://issues.apache.org/jira/browse/ARROW-9480
> Project: Apache Arrow
> Issue Type: Sub-task
> Components: Rust, Rust - DataFusion
> Reporter: Andy Grove
> Assignee: Andy Grove
> Priority: Major
>
> All DataFusion execution plan traits should require Send + Sync, to prepare for async support.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)