Skip to content

Support multi-partitions when compiling DataFusion to wasm target #15599

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

Open
Tracked by #13815
XiangpengHao opened this issue Apr 5, 2025 · 3 comments
Open
Tracked by #13815
Labels
enhancement New feature or request

Comments

@XiangpengHao
Copy link
Contributor

Is your feature request related to a problem or challenge?

This test case will fail (assuming #15595 is merged):

#[wasm_bindgen_test(unsupported = tokio::test)]
async fn test_multiple_partitions() {
    use futures::StreamExt;
    let ctx = SessionContext::new();
    let dummy_schema = Schema::new(vec![Field::new("a", DataType::Int64, false)]);
    let placeholder =
        datafusion_physical_plan::placeholder_row::PlaceholderRowExec::new(Arc::new(
            dummy_schema,
        ))
        .with_partitions(2);

    let task_ctx = ctx.task_ctx();
    let plan =
        datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec::new(
            Arc::new(placeholder),
        );
    let mut stream =
        datafusion_physical_plan::ExecutionPlan::execute(&plan, 0, task_ctx).unwrap();
    let batch = stream.next().await.unwrap().unwrap();
    assert_eq!(batch.num_rows(), 1);
    assert_eq!(batch.column(0).len(), 1);
}

To reproduce, you need to copy the test to datafusion/wasmtest/src/lib.rs and run:

wasm-pack test --headless --chrome

It fails with:

console.log div contained:
    panicked at /home/hao/coding/datafusion/datafusion/common-runtime/src/join_set.rs:66:20:
    there is no reactor running, must be called from the context of a Tokio 1.x runtime
    
    Stack:
    
    Error
        at http://127.0.0.1:39955/wasm-bindgen-test:556:21
        at logError (http://127.0.0.1:39955/wasm-bindgen-test:15:18)
        at imports.wbg.__wbg_new_78093c5bd701d017 (http://127.0.0.1:39955/wasm-bindgen-test:555:66)

However, if you change to with_partitions(1), the test will pass.

Related to #14478 (might be too advanced for a gsoc project though)
#13815 #13715 #13818

Describe the solution you'd like

It fails because it tries to span tokio tasks in browser, which does not have a tokio runner.

Instead, it should spawn the task to browser's event_loop, which imo is quite non-trivial.

This requires us to have a wasm-specific implementation of JoinSet, which spawns tasks to browser event loop rather than tokio runtime.

A deeper implication is that this also moves DataFusion towards a runtime-agnostic design. I personally believe this is a good thing, but I'm not sure if it is worth the effort.

Describe alternatives you've considered

If we do nothing, everything still works, just means that DataFusion on wasm can only use a single partition, thus single threaded.

Or we can wait and see if wasi becomes the standard and "DataFusion on wasm" is not wasm32-unknown-unknown, but rather wasm32-wasi, which should magically fix the problem. However, I expect this to take many years to become a reality.

(cc @alamb, this is the reason parquet-viewer stays single-threaded)

Additional context

Reference: https://github.com/cunarist/tokio-with-wasm

@XiangpengHao XiangpengHao added the enhancement New feature or request label Apr 5, 2025
@XiangpengHao
Copy link
Contributor Author

cc @waynexia, who might have thought about this

@alamb alamb changed the title Support multi-partitions in wasm target Support multi-partitions when compiling DataFusion to wasm target Apr 7, 2025
@alamb
Copy link
Contributor

alamb commented Apr 7, 2025

Thanks for the ping -- I added this ticket to the list on #13815 as well

@waynexia
Copy link
Member

waynexia commented Apr 9, 2025

I've run into this problem as well. I don't trace the latest progress of multi-thread support in WASM, but considering the real world WASM ecosystem, maybe we'll have to maintain (at least) two targets...

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

No branches or pull requests

3 participants