|
1 | 1 | # Getting Started |
2 | 2 |
|
3 | | -Rather than being opinionated about your setup and how you serve queries to users, |
4 | | -Distributed DataFusion allows you to plug in your own networking stack and spawn your own gRPC servers that act as |
5 | | -workers in the cluster. |
| 3 | +Think of this library as vanilla DataFusion, except that certain nodes execute their children on remote |
| 4 | +machines and retrieve data via the Arrow Flight protocol. |
| 5 | + |
| 6 | +This library aims to provide an experience as close as possible to vanilla DataFusion. |
| 7 | + |
| 8 | +## How to use Distributed DataFusion |
| 9 | + |
| 10 | +Rather than imposing constraints on your infrastructure or query serving patterns, Distributed DataFusion |
| 11 | +allows you to plug in your own networking stack and spawn your own gRPC servers that act as workers in the cluster. |
6 | 12 |
|
7 | 13 | This project heavily relies on the [Tonic](https://github.com/hyperium/tonic) ecosystem for the networking layer. |
8 | 14 | Users of this library are responsible for building their own Tonic server, adding the Arrow Flight distributed |
9 | | -DataFusion service to it and spawning it on a port so that it can be reached by other workers in the cluster. |
| 15 | +DataFusion service to it and spawning it on a port so that it can be reached by other workers in the cluster. A very |
| 16 | +basic example of this would be: |
10 | 17 |
|
11 | | -For a basic setup, all you need to do is to enrich your DataFusion `SessionStateBuilder` with the tools this project |
12 | | -ships: |
| 18 | +```rust |
| 19 | +#[tokio::main] |
| 20 | +async fn main() -> Result<(), Box<dyn Error>> { |
| 21 | + let worker = Worker::default(); |
13 | 22 |
|
14 | | -```rs |
15 | | - let state = SessionStateBuilder::new() |
16 | | -+ .with_distributed_worker_resolver(my_custom_worker_resolver) |
17 | | -+ .with_physical_optimizer_rule(Arc::new(DistributedPhysicalOptimizerRule)) |
18 | | - .build(); |
19 | | -``` |
| 23 | + Server::builder() |
| 24 | + .add_service(worker.into_flight_server()) |
| 25 | + .serve(SocketAddr::new(IpAddr::V4(Ipv4Addr::LOCALHOST), 8000)) |
| 26 | + .await?; |
20 | 27 |
|
21 | | -And the `my_custom_worker_resolver` variable should be an implementation of |
22 | | -the [WorkerResolverResolver](https://github.com/datafusion-contrib/datafusion-distributed/blob/main/src/networking/worker_resolver.rs) |
23 | | -trait, which tells Distributed DataFusion how to connect to other workers in the cluster. |
| 28 | + Ok(()) |
| 29 | +} |
| 30 | +``` |
24 | 31 |
|
25 | | -A very basic example of such an implementation that resolves workers in the localhost machine is: |
| 32 | +Distributed DataFusion requires knowledge of worker locations. Implement the `WorkerResolver` trait to provide |
| 33 | +this information. Here is a simple example of what this would look like with localhost workers: |
26 | 34 |
|
27 | 35 | ```rust |
28 | 36 | #[derive(Clone)] |
29 | 37 | struct LocalhostWorkerResolver { |
30 | 38 | ports: Vec<u16>, |
31 | 39 | } |
32 | 40 |
|
33 | | -#[async_trait] |
34 | | -impl ChannelResolver for LocalhostChannelResolver { |
| 41 | +impl WorkerResolver for LocalhostWorkerResolver { |
35 | 42 | fn get_urls(&self) -> Result<Vec<Url>, DataFusionError> { |
36 | | - Ok(self.ports.iter().map(|port| Url::parse(&format!("http://localhost:{port}")).unwrap()).collect()) |
| 43 | + Ok(self |
| 44 | + .ports |
| 45 | + .iter() |
| 46 | + .map(|port| Url::parse(&format!("http://localhost:{port}")).unwrap()) |
| 47 | + .collect()) |
37 | 48 | } |
38 | 49 | } |
39 | 50 | ``` |
40 | 51 |
|
41 | | -> NOTE: This example is not production-ready and is meant to showcase the basic concepts of the library. |
42 | | -
|
43 | | -This `WorkerResolver` implementation should resolve URLs of Distributed DataFusion workers, and it's |
44 | | -also the user of this library's responsibility to spawn a Tonic server that exposes the worker as an Arrow Flight |
45 | | -service using Tonic. |
| 52 | +Register both the `WorkerResolver` implementation and the `DistributedPhysicalOptimizerRule` in DataFusion's |
| 53 | +`SessionStateBuilder` to enable distributed query planning: |
46 | 54 |
|
47 | | -A basic example of such a server is: |
48 | | - |
49 | | -```rust |
50 | | -#[tokio::main] |
51 | | -async fn main() -> Result<(), Box<dyn Error>> { |
52 | | - let endpoint = Worker::default(); |
53 | | - |
54 | | - Server::builder() |
55 | | - .add_service(endpoint.into_flight_server()) |
56 | | - .serve(SocketAddr::new(IpAddr::V4(Ipv4Addr::LOCALHOST), 8000)) |
57 | | - .await?; |
58 | | - |
59 | | - Ok(()) |
| 55 | +```rs |
| 56 | +let localhost_worker_resolver = LocalhostWorkerResolver { |
| 57 | + ports: vec![8000, 8001, 8002] |
60 | 58 | } |
61 | | -``` |
62 | 59 |
|
63 | | -## Next steps |
| 60 | +let state = SessionStateBuilder::new() |
| 61 | + .with_distributed_worker_resolver(localhost_worker_resolver) |
| 62 | + .with_physical_optimizer_rule(Arc::new(DistributedPhysicalOptimizerRule)) |
| 63 | + .build(); |
64 | 64 |
|
65 | | -The next two sections of this guide will walk you through tailoring the library's traits to your own needs: |
| 65 | +let ctx = SessionContext::from(state); |
| 66 | +``` |
66 | 67 |
|
67 | | -- [Build your own WorkerResolver](worker-resolver.md) |
68 | | -- [Build your own ChannelResolver](channel-resolver.md) |
69 | | -- [Build your own TaskEstimator](task-estimator.md) |
70 | | -- [Build your own distributed DataFusion Worker](worker.md) |
| 68 | +This will leave a DataFusion `SessionContext` ready for executing distributed queries. |
71 | 69 |
|
72 | | -Here are some other resources in the codebase: |
| 70 | +> NOTE: This example is not production-ready and is meant to showcase the basic concepts of the library. |
73 | 71 |
|
74 | | -- [In-memory cluster example](https://github.com/datafusion-contrib/datafusion-distributed/blob/main/examples/in_memory.md) |
75 | | -- [Localhost cluster example](https://github.com/datafusion-contrib/datafusion-distributed/blob/main/examples/localhost.md) |
| 72 | +## Next steps |
76 | 73 |
|
77 | | -A more advanced example can be found in the benchmarks that use a cluster of distributed DataFusion workers |
78 | | -deployed on AWS EC2 machines: |
| 74 | +Depending on your needs, your setup can get more complicated, for example: |
79 | 75 |
|
80 | | -- [AWS EC2 based cluster example](https://github.com/datafusion-contrib/datafusion-distributed/blob/main/benchmarks/cdk/bin/worker.rs) |
| 76 | +- You may want to resolve worker URLs dynamically using the Kubernetes API. |
| 77 | +- You may want to wrap the Arrow Flight clients that connect workers with an observability layer. |
| 78 | +- You may want to be able to execute your own custom ExecutionPlans in a distributed manner. |
| 79 | +- etc... |
81 | 80 |
|
82 | | -Each feature in the project is showcased and tested in its own isolated integration test, so it's recommended to |
83 | | -review those for a better understanding of how specific features work: |
| 81 | +To learn how to do all that, it's recommended to: |
84 | 82 |
|
85 | | -- [Pass your own ConfigExtension implementations across network boundaries](https://github.com/datafusion-contrib/datafusion-distributed/blob/main/tests/custom_config_extension.rs) |
86 | | -- [Provide custom protobuf codecs for your own nodes](https://github.com/datafusion-contrib/datafusion-distributed/blob/main/tests/custom_extension_codec.rs) |
87 | | -- Provide a custom TaskEstimator for controlling the amount of parallelism (coming soon) |
| 83 | +- [Continue reading this guide](worker.md) |
| 84 | +- [Look at examples in the project](https://github.com/datafusion-contrib/datafusion-distributed/tree/main/examples) |
| 85 | +- [Look at the integration tests for finer grained examples](https://github.com/datafusion-contrib/datafusion-distributed/tree/main/tests) |
88 | 86 |
|
0 commit comments