From 2fd4f80d22ae3fc313d92d8afcca383e55e575ad Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 8 Apr 2021 16:20:33 -0600 Subject: [PATCH] ARROW-11982: [Rust] Donate Ballista Distributed Compute Platform MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This PR contains a donation of the Ballista Distributed Compute platform. We need to meet the following conditions before merging this PR: 1. Pass a vote on the Arrow dev@ mailing list (Completed on 3/25/2021) 2. Pass the IP clearance process This PR is based on the contents of Ballista main branch at commit hash [9373241314bb9b01c639aa51c92159611fa42d3e](https://github.com/ballista-compute/ballista/commit/9373241314bb9b01c639aa51c92159611fa42d3e) and contains work from the following contributors: ``` $ git shortlog -n -s -- rust 231 Andy Grove 27 Ximo Guanter 7 QP Hou 6 Boaz Berman 5 Daniël Heres 4 Jack Chan 4 Patrick More 3 Ben Cohen 3 Hendrik Makait 3 Jorge Leitao 2 Hamish Nicholson 2 Tanapol Prucksakorn 1 Andrew Fitzgerald 1 Dyqer 1 Enrico Risa 1 Jeremy Mei(梅杰) 1 K.I. (Dennis) Jung 1 Kyle Prifogle 1 Matthijs Brobbel 1 Nikita Lapkov 1 Sieu TSoi 1 Son 1 Viktor Gal 1 Yeshwanth Kumar 1 jschmitz28 $ git shortlog -n -s -- dev 45 Andy Grove 3 Ximo Guanter 2 Jorge Leitao $ git shortlog -n -s -- docker 30 Andy Grove 6 Ximo Guanter 2 Daniël Heres 1 Ben Sully 1 Hamish Nicholson 1 Jorge Leitao 1 Rick Richardson $ git shortlog -n -s -- docs 37 Andy Grove 2 K.I. (Dennis) Jung 1 Hamish Nicholson 1 Jorge Leitao 1 Ximo Guanter ``` Closes #9723 from andygrove/donate-ballista Authored-by: Andy Grove Signed-off-by: Andy Grove --- dev/release/rat_exclude_files.txt | 2 + rust/Cargo.toml | 2 +- rust/ballista/dev/build-rust-base.sh | 21 + rust/ballista/dev/build-rust.sh | 24 + rust/ballista/dev/integration-tests.sh | 28 + rust/ballista/docker/README.md | 29 + rust/ballista/docker/rust-base.dockerfile | 99 ++ rust/ballista/docker/rust.dockerfile | 71 + rust/ballista/docs/README.md | 37 + rust/ballista/docs/architecture.md | 75 + rust/ballista/docs/dev-env-rust.md | 38 + rust/ballista/docs/images/query-execution.png | Bin 0 -> 11378 bytes rust/ballista/docs/integration-testing.md | 32 + rust/ballista/docs/release-process.md | 68 + rust/ballista/docs/rust-docker.md | 66 + rust/ballista/docs/user-guide/.gitignore | 2 + rust/ballista/docs/user-guide/README.md | 36 + rust/ballista/docs/user-guide/book.toml | 23 + rust/ballista/docs/user-guide/src/SUMMARY.md | 30 + .../docs/user-guide/src/client-rust.md | 22 + rust/ballista/docs/user-guide/src/clients.md | 22 + .../docs/user-guide/src/configuration.md | 32 + .../docs/user-guide/src/deployment.md | 26 + .../docs/user-guide/src/docker-compose.md | 55 + rust/ballista/docs/user-guide/src/faq.md | 31 + .../src/img/ballista-architecture.png | Bin 0 -> 21225 bytes .../docs/user-guide/src/introduction.md | 52 + .../docs/user-guide/src/kubernetes.md | 216 +++ .../docs/user-guide/src/standalone.md | 92 ++ rust/ballista/rust/.dockerignore | 23 + rust/ballista/rust/.gitignore | 2 + rust/ballista/rust/Cargo.toml | 30 + .../rust/benchmarks/tpch/.dockerignore | 25 + rust/ballista/rust/benchmarks/tpch/.gitignore | 1 + rust/ballista/rust/benchmarks/tpch/Cargo.toml | 37 + rust/ballista/rust/benchmarks/tpch/README.md | 103 ++ .../rust/benchmarks/tpch/docker-compose.yaml | 62 + .../rust/benchmarks/tpch/entrypoint.sh | 22 + .../rust/benchmarks/tpch/queries/q1.sql | 21 + .../rust/benchmarks/tpch/queries/q10.sql | 31 + .../rust/benchmarks/tpch/queries/q11.sql | 27 + .../rust/benchmarks/tpch/queries/q12.sql | 30 + .../rust/benchmarks/tpch/queries/q13.sql | 20 + .../rust/benchmarks/tpch/queries/q14.sql | 13 + .../rust/benchmarks/tpch/queries/q16.sql | 30 + .../rust/benchmarks/tpch/queries/q17.sql | 17 + .../rust/benchmarks/tpch/queries/q18.sql | 32 + .../rust/benchmarks/tpch/queries/q19.sql | 35 + .../rust/benchmarks/tpch/queries/q2.sql | 43 + .../rust/benchmarks/tpch/queries/q20.sql | 37 + .../rust/benchmarks/tpch/queries/q21.sql | 39 + .../rust/benchmarks/tpch/queries/q22.sql | 37 + .../rust/benchmarks/tpch/queries/q3.sql | 22 + .../rust/benchmarks/tpch/queries/q4.sql | 21 + .../rust/benchmarks/tpch/queries/q5.sql | 24 + .../rust/benchmarks/tpch/queries/q6.sql | 9 + .../rust/benchmarks/tpch/queries/q7.sql | 39 + .../rust/benchmarks/tpch/queries/q8.sql | 37 + .../rust/benchmarks/tpch/queries/q9.sql | 32 + rust/ballista/rust/benchmarks/tpch/run.sh | 25 + .../ballista/rust/benchmarks/tpch/src/main.rs | 360 +++++ .../ballista/rust/benchmarks/tpch/tpch-gen.sh | 33 + .../rust/benchmarks/tpch/tpchgen.dockerfile | 32 + rust/ballista/rust/client/Cargo.toml | 34 + rust/ballista/rust/client/README.md | 22 + .../rust/client/src/columnar_batch.rs | 167 +++ rust/ballista/rust/client/src/context.rs | 372 +++++ rust/ballista/rust/client/src/lib.rs | 20 + rust/ballista/rust/client/src/prelude.rs | 23 + rust/ballista/rust/core/Cargo.toml | 49 + rust/ballista/rust/core/README.md | 21 + rust/ballista/rust/core/build.rs | 26 + rust/ballista/rust/core/proto/ballista.proto | 818 +++++++++++ rust/ballista/rust/core/src/client.rs | 211 +++ rust/ballista/rust/core/src/datasource.rs | 71 + rust/ballista/rust/core/src/error.rs | 166 +++ .../rust/core/src/execution_plans/mod.rs | 27 + .../core/src/execution_plans/query_stage.rs | 88 ++ .../src/execution_plans/shuffle_reader.rs | 103 ++ .../src/execution_plans/unresolved_shuffle.rs | 96 ++ rust/ballista/rust/core/src/lib.rs | 34 + rust/ballista/rust/core/src/memory_stream.rs | 93 ++ .../core/src/serde/logical_plan/from_proto.rs | 1060 +++++++++++++++ .../rust/core/src/serde/logical_plan/mod.rs | 923 +++++++++++++ .../core/src/serde/logical_plan/to_proto.rs | 1209 +++++++++++++++++ rust/ballista/rust/core/src/serde/mod.rs | 69 + .../src/serde/physical_plan/from_proto.rs | 367 +++++ .../rust/core/src/serde/physical_plan/mod.rs | 173 +++ .../core/src/serde/physical_plan/to_proto.rs | 525 +++++++ .../core/src/serde/scheduler/from_proto.rs | 123 ++ .../rust/core/src/serde/scheduler/mod.rs | 256 ++++ .../rust/core/src/serde/scheduler/to_proto.rs | 90 ++ rust/ballista/rust/core/src/utils.rs | 293 ++++ rust/ballista/rust/executor/Cargo.toml | 59 + rust/ballista/rust/executor/README.md | 31 + rust/ballista/rust/executor/build.rs | 24 + .../examples/example_executor_config.toml | 22 + .../rust/executor/executor_config_spec.toml | 79 ++ rust/ballista/rust/executor/src/collect.rs | 124 ++ .../rust/executor/src/execution_loop.rs | 166 +++ .../rust/executor/src/flight_service.rs | 357 +++++ rust/ballista/rust/executor/src/lib.rs | 52 + rust/ballista/rust/executor/src/main.rs | 168 +++ rust/ballista/rust/scheduler/Cargo.toml | 61 + rust/ballista/rust/scheduler/README.md | 32 + rust/ballista/rust/scheduler/build.rs | 24 + .../rust/scheduler/scheduler_config_spec.toml | 60 + rust/ballista/rust/scheduler/src/lib.rs | 495 +++++++ rust/ballista/rust/scheduler/src/main.rs | 120 ++ rust/ballista/rust/scheduler/src/planner.rs | 473 +++++++ .../ballista/rust/scheduler/src/state/etcd.rs | 116 ++ rust/ballista/rust/scheduler/src/state/mod.rs | 792 +++++++++++ .../rust/scheduler/src/state/standalone.rs | 156 +++ .../ballista/rust/scheduler/src/test_utils.rs | 136 ++ .../scheduler/testdata/customer/customer.tbl | 10 + .../testdata/lineitem/partition0.tbl | 10 + .../testdata/lineitem/partition1.tbl | 10 + .../rust/scheduler/testdata/nation/nation.tbl | 10 + .../rust/scheduler/testdata/orders/orders.tbl | 10 + .../rust/scheduler/testdata/part/part.tbl | 10 + .../scheduler/testdata/partsupp/partsupp.tbl | 10 + .../rust/scheduler/testdata/region/region.tbl | 5 + .../scheduler/testdata/supplier/supplier.tbl | 10 + 123 files changed, 13742 insertions(+), 1 deletion(-) create mode 100755 rust/ballista/dev/build-rust-base.sh create mode 100755 rust/ballista/dev/build-rust.sh create mode 100755 rust/ballista/dev/integration-tests.sh create mode 100644 rust/ballista/docker/README.md create mode 100644 rust/ballista/docker/rust-base.dockerfile create mode 100644 rust/ballista/docker/rust.dockerfile create mode 100644 rust/ballista/docs/README.md create mode 100644 rust/ballista/docs/architecture.md create mode 100644 rust/ballista/docs/dev-env-rust.md create mode 100644 rust/ballista/docs/images/query-execution.png create mode 100644 rust/ballista/docs/integration-testing.md create mode 100644 rust/ballista/docs/release-process.md create mode 100644 rust/ballista/docs/rust-docker.md create mode 100644 rust/ballista/docs/user-guide/.gitignore create mode 100644 rust/ballista/docs/user-guide/README.md create mode 100644 rust/ballista/docs/user-guide/book.toml create mode 100644 rust/ballista/docs/user-guide/src/SUMMARY.md create mode 100644 rust/ballista/docs/user-guide/src/client-rust.md create mode 100644 rust/ballista/docs/user-guide/src/clients.md create mode 100644 rust/ballista/docs/user-guide/src/configuration.md create mode 100644 rust/ballista/docs/user-guide/src/deployment.md create mode 100644 rust/ballista/docs/user-guide/src/docker-compose.md create mode 100644 rust/ballista/docs/user-guide/src/faq.md create mode 100644 rust/ballista/docs/user-guide/src/img/ballista-architecture.png create mode 100644 rust/ballista/docs/user-guide/src/introduction.md create mode 100644 rust/ballista/docs/user-guide/src/kubernetes.md create mode 100644 rust/ballista/docs/user-guide/src/standalone.md create mode 100644 rust/ballista/rust/.dockerignore create mode 100644 rust/ballista/rust/.gitignore create mode 100644 rust/ballista/rust/Cargo.toml create mode 100644 rust/ballista/rust/benchmarks/tpch/.dockerignore create mode 100644 rust/ballista/rust/benchmarks/tpch/.gitignore create mode 100644 rust/ballista/rust/benchmarks/tpch/Cargo.toml create mode 100644 rust/ballista/rust/benchmarks/tpch/README.md create mode 100644 rust/ballista/rust/benchmarks/tpch/docker-compose.yaml create mode 100755 rust/ballista/rust/benchmarks/tpch/entrypoint.sh create mode 100644 rust/ballista/rust/benchmarks/tpch/queries/q1.sql create mode 100644 rust/ballista/rust/benchmarks/tpch/queries/q10.sql create mode 100644 rust/ballista/rust/benchmarks/tpch/queries/q11.sql create mode 100644 rust/ballista/rust/benchmarks/tpch/queries/q12.sql create mode 100644 rust/ballista/rust/benchmarks/tpch/queries/q13.sql create mode 100644 rust/ballista/rust/benchmarks/tpch/queries/q14.sql create mode 100644 rust/ballista/rust/benchmarks/tpch/queries/q16.sql create mode 100644 rust/ballista/rust/benchmarks/tpch/queries/q17.sql create mode 100644 rust/ballista/rust/benchmarks/tpch/queries/q18.sql create mode 100644 rust/ballista/rust/benchmarks/tpch/queries/q19.sql create mode 100644 rust/ballista/rust/benchmarks/tpch/queries/q2.sql create mode 100644 rust/ballista/rust/benchmarks/tpch/queries/q20.sql create mode 100644 rust/ballista/rust/benchmarks/tpch/queries/q21.sql create mode 100644 rust/ballista/rust/benchmarks/tpch/queries/q22.sql create mode 100644 rust/ballista/rust/benchmarks/tpch/queries/q3.sql create mode 100644 rust/ballista/rust/benchmarks/tpch/queries/q4.sql create mode 100644 rust/ballista/rust/benchmarks/tpch/queries/q5.sql create mode 100644 rust/ballista/rust/benchmarks/tpch/queries/q6.sql create mode 100644 rust/ballista/rust/benchmarks/tpch/queries/q7.sql create mode 100644 rust/ballista/rust/benchmarks/tpch/queries/q8.sql create mode 100644 rust/ballista/rust/benchmarks/tpch/queries/q9.sql create mode 100755 rust/ballista/rust/benchmarks/tpch/run.sh create mode 100644 rust/ballista/rust/benchmarks/tpch/src/main.rs create mode 100755 rust/ballista/rust/benchmarks/tpch/tpch-gen.sh create mode 100644 rust/ballista/rust/benchmarks/tpch/tpchgen.dockerfile create mode 100644 rust/ballista/rust/client/Cargo.toml create mode 100644 rust/ballista/rust/client/README.md create mode 100644 rust/ballista/rust/client/src/columnar_batch.rs create mode 100644 rust/ballista/rust/client/src/context.rs create mode 100644 rust/ballista/rust/client/src/lib.rs create mode 100644 rust/ballista/rust/client/src/prelude.rs create mode 100644 rust/ballista/rust/core/Cargo.toml create mode 100644 rust/ballista/rust/core/README.md create mode 100644 rust/ballista/rust/core/build.rs create mode 100644 rust/ballista/rust/core/proto/ballista.proto create mode 100644 rust/ballista/rust/core/src/client.rs create mode 100644 rust/ballista/rust/core/src/datasource.rs create mode 100644 rust/ballista/rust/core/src/error.rs create mode 100644 rust/ballista/rust/core/src/execution_plans/mod.rs create mode 100644 rust/ballista/rust/core/src/execution_plans/query_stage.rs create mode 100644 rust/ballista/rust/core/src/execution_plans/shuffle_reader.rs create mode 100644 rust/ballista/rust/core/src/execution_plans/unresolved_shuffle.rs create mode 100644 rust/ballista/rust/core/src/lib.rs create mode 100644 rust/ballista/rust/core/src/memory_stream.rs create mode 100644 rust/ballista/rust/core/src/serde/logical_plan/from_proto.rs create mode 100644 rust/ballista/rust/core/src/serde/logical_plan/mod.rs create mode 100644 rust/ballista/rust/core/src/serde/logical_plan/to_proto.rs create mode 100644 rust/ballista/rust/core/src/serde/mod.rs create mode 100644 rust/ballista/rust/core/src/serde/physical_plan/from_proto.rs create mode 100644 rust/ballista/rust/core/src/serde/physical_plan/mod.rs create mode 100644 rust/ballista/rust/core/src/serde/physical_plan/to_proto.rs create mode 100644 rust/ballista/rust/core/src/serde/scheduler/from_proto.rs create mode 100644 rust/ballista/rust/core/src/serde/scheduler/mod.rs create mode 100644 rust/ballista/rust/core/src/serde/scheduler/to_proto.rs create mode 100644 rust/ballista/rust/core/src/utils.rs create mode 100644 rust/ballista/rust/executor/Cargo.toml create mode 100644 rust/ballista/rust/executor/README.md create mode 100644 rust/ballista/rust/executor/build.rs create mode 100644 rust/ballista/rust/executor/examples/example_executor_config.toml create mode 100644 rust/ballista/rust/executor/executor_config_spec.toml create mode 100644 rust/ballista/rust/executor/src/collect.rs create mode 100644 rust/ballista/rust/executor/src/execution_loop.rs create mode 100644 rust/ballista/rust/executor/src/flight_service.rs create mode 100644 rust/ballista/rust/executor/src/lib.rs create mode 100644 rust/ballista/rust/executor/src/main.rs create mode 100644 rust/ballista/rust/scheduler/Cargo.toml create mode 100644 rust/ballista/rust/scheduler/README.md create mode 100644 rust/ballista/rust/scheduler/build.rs create mode 100644 rust/ballista/rust/scheduler/scheduler_config_spec.toml create mode 100644 rust/ballista/rust/scheduler/src/lib.rs create mode 100644 rust/ballista/rust/scheduler/src/main.rs create mode 100644 rust/ballista/rust/scheduler/src/planner.rs create mode 100644 rust/ballista/rust/scheduler/src/state/etcd.rs create mode 100644 rust/ballista/rust/scheduler/src/state/mod.rs create mode 100644 rust/ballista/rust/scheduler/src/state/standalone.rs create mode 100644 rust/ballista/rust/scheduler/src/test_utils.rs create mode 100644 rust/ballista/rust/scheduler/testdata/customer/customer.tbl create mode 100644 rust/ballista/rust/scheduler/testdata/lineitem/partition0.tbl create mode 100644 rust/ballista/rust/scheduler/testdata/lineitem/partition1.tbl create mode 100644 rust/ballista/rust/scheduler/testdata/nation/nation.tbl create mode 100644 rust/ballista/rust/scheduler/testdata/orders/orders.tbl create mode 100644 rust/ballista/rust/scheduler/testdata/part/part.tbl create mode 100644 rust/ballista/rust/scheduler/testdata/partsupp/partsupp.tbl create mode 100644 rust/ballista/rust/scheduler/testdata/region/region.tbl create mode 100644 rust/ballista/rust/scheduler/testdata/supplier/supplier.tbl diff --git a/dev/release/rat_exclude_files.txt b/dev/release/rat_exclude_files.txt index c2e22abd39c8a..e943dadf29d04 100644 --- a/dev/release/rat_exclude_files.txt +++ b/dev/release/rat_exclude_files.txt @@ -195,3 +195,5 @@ julia/Arrow/docs/mkdocs.yml julia/Arrow/docs/src/index.md julia/Arrow/docs/src/manual.md julia/Arrow/docs/src/reference.md +rust/ballista/rust/benchmarks/tpch/queries/q*.sql +rust/ballista/rust/scheduler/testdata/* diff --git a/rust/Cargo.toml b/rust/Cargo.toml index c29900429b274..de26f87c77846 100644 --- a/rust/Cargo.toml +++ b/rust/Cargo.toml @@ -31,4 +31,4 @@ members = [ # this package is excluded because it requires different compilation flags, thereby significantly changing # how it is compiled within the workspace, causing the whole workspace to be compiled from scratch # this way, this is a stand-alone package that compiles independently of the others. -exclude = ["arrow-pyarrow-integration-testing"] +exclude = ["arrow-pyarrow-integration-testing", "ballista"] diff --git a/rust/ballista/dev/build-rust-base.sh b/rust/ballista/dev/build-rust-base.sh new file mode 100755 index 0000000000000..ee4b32c8e690a --- /dev/null +++ b/rust/ballista/dev/build-rust-base.sh @@ -0,0 +1,21 @@ +#!/bin/bash + +# 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_VERSION=0.4.2-SNAPSHOT +set -e +docker build -t ballistacompute/rust-base:$BALLISTA_VERSION -f docker/rust-base.dockerfile . diff --git a/rust/ballista/dev/build-rust.sh b/rust/ballista/dev/build-rust.sh new file mode 100755 index 0000000000000..1916f8efbefb1 --- /dev/null +++ b/rust/ballista/dev/build-rust.sh @@ -0,0 +1,24 @@ +#!/bin/bash + +# 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_VERSION=0.4.2-SNAPSHOT + +set -e + +docker build -t ballistacompute/ballista-rust:$BALLISTA_VERSION -f docker/rust.dockerfile . diff --git a/rust/ballista/dev/integration-tests.sh b/rust/ballista/dev/integration-tests.sh new file mode 100755 index 0000000000000..cc34a5ce91f53 --- /dev/null +++ b/rust/ballista/dev/integration-tests.sh @@ -0,0 +1,28 @@ +#!/bin/bash + +# 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. +set -e +./dev/build-rust.sh +pushd rust/benchmarks/tpch +./tpch-gen.sh + +docker-compose up -d +docker-compose run ballista-client ./run.sh +docker-compose down + +popd diff --git a/rust/ballista/docker/README.md b/rust/ballista/docker/README.md new file mode 100644 index 0000000000000..8417d04c49220 --- /dev/null +++ b/rust/ballista/docker/README.md @@ -0,0 +1,29 @@ + + +# Ballista Docker Images + +Pre-built docker images are available from [Docker Hub](https://hub.docker.com/orgs/ballistacompute/repositories) but here are the commands to build the images from source. + +Run these commands from the root directory of the project. + +```bash +./dev/build-all.sh +``` + diff --git a/rust/ballista/docker/rust-base.dockerfile b/rust/ballista/docker/rust-base.dockerfile new file mode 100644 index 0000000000000..4519225d2197c --- /dev/null +++ b/rust/ballista/docker/rust-base.dockerfile @@ -0,0 +1,99 @@ +# 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. + +# Turn .dockerignore to .dockerallow by excluding everything and explicitly +# allowing specific files and directories. This enables us to quickly add +# dependency files to the docker content without scanning the whole directory. +# This setup requires to all of our docker containers have arrow's source +# as a mounted directory. + + +# Base image extends debian:buster-slim +FROM rust:1.49.0-buster AS builder + +RUN apt update && apt -y install musl musl-dev musl-tools libssl-dev openssl + +#NOTE: the following was copied from https://github.com/emk/rust-musl-builder/blob/master/Dockerfile under Apache 2.0 license + +# The OpenSSL version to use. We parameterize this because many Rust +# projects will fail to build with 1.1. +#ARG OPENSSL_VERSION=1.0.2r +ARG OPENSSL_VERSION=1.1.1b + +# Build a static library version of OpenSSL using musl-libc. This is needed by +# the popular Rust `hyper` crate. +# +# We point /usr/local/musl/include/linux at some Linux kernel headers (not +# necessarily the right ones) in an effort to compile OpenSSL 1.1's "engine" +# component. It's possible that this will cause bizarre and terrible things to +# happen. There may be "sanitized" header +RUN echo "Building OpenSSL" && \ + ls /usr/include/linux && \ + mkdir -p /usr/local/musl/include && \ + ln -s /usr/include/linux /usr/local/musl/include/linux && \ + ln -s /usr/include/x86_64-linux-gnu/asm /usr/local/musl/include/asm && \ + ln -s /usr/include/asm-generic /usr/local/musl/include/asm-generic && \ + cd /tmp && \ + curl -LO "https://www.openssl.org/source/openssl-$OPENSSL_VERSION.tar.gz" && \ + tar xvzf "openssl-$OPENSSL_VERSION.tar.gz" && cd "openssl-$OPENSSL_VERSION" && \ + env CC=musl-gcc ./Configure no-shared no-zlib -fPIC --prefix=/usr/local/musl -DOPENSSL_NO_SECURE_MEMORY linux-x86_64 && \ + env C_INCLUDE_PATH=/usr/local/musl/include/ make depend && \ + env C_INCLUDE_PATH=/usr/local/musl/include/ make && \ + make install && \ + rm /usr/local/musl/include/linux /usr/local/musl/include/asm /usr/local/musl/include/asm-generic && \ + rm -r /tmp/* + +RUN echo "Building zlib" && \ + cd /tmp && \ + ZLIB_VERSION=1.2.11 && \ + curl -LO "http://zlib.net/zlib-$ZLIB_VERSION.tar.gz" && \ + tar xzf "zlib-$ZLIB_VERSION.tar.gz" && cd "zlib-$ZLIB_VERSION" && \ + CC=musl-gcc ./configure --static --prefix=/usr/local/musl && \ + make && make install && \ + rm -r /tmp/* + +RUN echo "Building libpq" && \ + cd /tmp && \ + POSTGRESQL_VERSION=11.2 && \ + curl -LO "https://ftp.postgresql.org/pub/source/v$POSTGRESQL_VERSION/postgresql-$POSTGRESQL_VERSION.tar.gz" && \ + tar xzf "postgresql-$POSTGRESQL_VERSION.tar.gz" && cd "postgresql-$POSTGRESQL_VERSION" && \ + CC=musl-gcc CPPFLAGS=-I/usr/local/musl/include LDFLAGS=-L/usr/local/musl/lib ./configure --with-openssl --without-readline --prefix=/usr/local/musl && \ + cd src/interfaces/libpq && make all-static-lib && make install-lib-static && \ + cd ../../bin/pg_config && make && make install && \ + rm -r /tmp/* + +ENV OPENSSL_DIR=/usr/local/musl/ \ + OPENSSL_INCLUDE_DIR=/usr/local/musl/include/ \ + DEP_OPENSSL_INCLUDE=/usr/local/musl/include/ \ + OPENSSL_LIB_DIR=/usr/local/musl/lib/ \ + OPENSSL_STATIC=1 \ + PQ_LIB_STATIC_X86_64_UNKNOWN_LINUX_MUSL=1 \ + PG_CONFIG_X86_64_UNKNOWN_LINUX_GNU=/usr/bin/pg_config \ + PKG_CONFIG_ALLOW_CROSS=true \ + PKG_CONFIG_ALL_STATIC=true \ + LIBZ_SYS_STATIC=1 \ + TARGET=musl + +# The content copied mentioned in the NOTE above ends here. + +## Download the target for static linking. +RUN rustup target add x86_64-unknown-linux-musl +RUN cargo install cargo-build-deps + +# prepare toolchain +RUN rustup update && \ + rustup component add rustfmt \ No newline at end of file diff --git a/rust/ballista/docker/rust.dockerfile b/rust/ballista/docker/rust.dockerfile new file mode 100644 index 0000000000000..8b06af3dc78fc --- /dev/null +++ b/rust/ballista/docker/rust.dockerfile @@ -0,0 +1,71 @@ +# 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. + +# Turn .dockerignore to .dockerallow by excluding everything and explicitly +# allowing specific files and directories. This enables us to quickly add +# dependency files to the docker content without scanning the whole directory. +# This setup requires to all of our docker containers have arrow's source +# as a mounted directory. + +ARG RELEASE_FLAG=--release +FROM ballistacompute/rust-base:0.4.0-20210213 AS base +WORKDIR /tmp/ballista +RUN apt-get -y install cmake +RUN cargo install cargo-chef + +FROM base as planner +COPY rust . +RUN cargo chef prepare --recipe-path recipe.json + +FROM base as cacher +COPY --from=planner /tmp/ballista/recipe.json recipe.json +RUN cargo chef cook $RELEASE_FLAG --recipe-path recipe.json + +FROM base as builder +COPY rust . +COPY --from=cacher /tmp/ballista/target target +ARG RELEASE_FLAG=--release + +# force build.rs to run to generate configure_me code. +ENV FORCE_REBUILD='true' +RUN cargo build $RELEASE_FLAG + +# put the executor on /executor (need to be copied from different places depending on FLAG) +ENV RELEASE_FLAG=${RELEASE_FLAG} +RUN if [ -z "$RELEASE_FLAG" ]; then mv /tmp/ballista/target/debug/ballista-executor /executor; else mv /tmp/ballista/target/release/ballista-executor /executor; fi + +# put the scheduler on /scheduler (need to be copied from different places depending on FLAG) +ENV RELEASE_FLAG=${RELEASE_FLAG} +RUN if [ -z "$RELEASE_FLAG" ]; then mv /tmp/ballista/target/debug/ballista-scheduler /scheduler; else mv /tmp/ballista/target/release/ballista-scheduler /scheduler; fi + +# put the tpch on /tpch (need to be copied from different places depending on FLAG) +ENV RELEASE_FLAG=${RELEASE_FLAG} +RUN if [ -z "$RELEASE_FLAG" ]; then mv /tmp/ballista/target/debug/tpch /tpch; else mv /tmp/ballista/target/release/tpch /tpch; fi + +# Copy the binary into a new container for a smaller docker image +FROM ballistacompute/rust-base:0.4.0-20210213 + +COPY --from=builder /executor / + +COPY --from=builder /scheduler / + +COPY --from=builder /tpch / + +ENV RUST_LOG=info +ENV RUST_BACKTRACE=full + +CMD ["/executor", "--local"] diff --git a/rust/ballista/docs/README.md b/rust/ballista/docs/README.md new file mode 100644 index 0000000000000..44c831d37800d --- /dev/null +++ b/rust/ballista/docs/README.md @@ -0,0 +1,37 @@ + +# Ballista Developer Documentation + +This directory contains documentation for developers that are contributing to Ballista. If you are looking for +end-user documentation for a published release, please start with the +[Ballista User Guide](https://ballistacompute.org/docs/) instead. + +## Architecture & Design + +- Read the [Architecture Overview](architecture.md) to get an understanding of the scheduler and executor + processes and how distributed query execution works. + +## Build, Test, Release + +- Setting up a [Rust development environment](dev-env-rust.md). +- Setting up a [Java development environment](dev-env-jvm.md). +- Notes on building [Rust docker images](rust-docker.md) +- [Integration Testing](integration-testing.md) +- [Release process](release-process.md) + diff --git a/rust/ballista/docs/architecture.md b/rust/ballista/docs/architecture.md new file mode 100644 index 0000000000000..a73b53a087016 --- /dev/null +++ b/rust/ballista/docs/architecture.md @@ -0,0 +1,75 @@ + +# Ballista Architecture + +## Overview + +Ballista allows queries to be executed in a distributed cluster. A cluster consists of one or +more scheduler processes and one or more executor processes. See the following sections in this document for more +details about these components. + +The scheduler accepts logical query plans and translates them into physical query plans using DataFusion and then +runs a secondary planning/optimization process to translate the physical query plan into a distributed physical +query plan. + +This process breaks a query down into a number of query stages that can be executed independently. There are +dependencies between query stages and these dependencies form a directionally-acyclic graph (DAG) because a query +stage cannot start until its child query stages have completed. + +Each query stage has one or more partitions that can be processed in parallel by the available +executors in the cluster. This is the basic unit of scalability in Ballista. + +The following diagram shows the flow of requests and responses between the client, scheduler, and executor +processes. + +![Query Execution Flow](images/query-execution.png) + +## Scheduler Process + +The scheduler process implements a gRPC interface (defined in +[ballista.proto](../rust/ballista/proto/ballista.proto)). The interface provides the following methods: + +| Method | Description | +|----------------------|----------------------------------------------------------------------| +| ExecuteQuery | Submit a logical query plan or SQL query for execution | +| GetExecutorsMetadata | Retrieves a list of executors that have registered with a scheduler | +| GetFileMetadata | Retrieve metadata about files available in the cluster file system | +| GetJobStatus | Get the status of a submitted query | +| RegisterExecutor | Executors call this method to register themselves with the scheduler | + +The scheduler can run in standalone mode, or can be run in clustered mode using etcd as backing store for state. + +## Executor Process + +The executor process implements the Apache Arrow Flight gRPC interface and is responsible for: + +- Executing query stages and persisting the results to disk in Apache Arrow IPC Format +- Making query stage results available as Flights so that they can be retrieved by other executors as well as by + clients + +## Rust Client + +The Rust client provides a DataFrame API that is a thin wrapper around the DataFusion DataFrame and provides +the means for a client to build a query plan for execution. + +The client executes the query plan by submitting an `ExecuteLogicalPlan` request to the scheduler and then calls +`GetJobStatus` to check for completion. On completion, the client receives a list of locations for the Flights +containing the results for the query and will then connect to the appropriate executor processes to retrieve +those results. + diff --git a/rust/ballista/docs/dev-env-rust.md b/rust/ballista/docs/dev-env-rust.md new file mode 100644 index 0000000000000..bf50c9d9c9137 --- /dev/null +++ b/rust/ballista/docs/dev-env-rust.md @@ -0,0 +1,38 @@ + +# Setting up a Rust development environment + +You will need a standard Rust development environment. The easiest way to achieve this is by using rustup: https://rustup.rs/ + +## Install OpenSSL + +Follow instructions for [setting up OpenSSL](https://docs.rs/openssl/0.10.28/openssl/). For Ubuntu users, the following +command works. + +```bash +sudo apt-get install pkg-config libssl-dev +``` + +## Install CMake + +You'll need cmake in order to compile some of ballista's dependencies. Ubuntu users can use the following command: + +```bash +sudo apt-get install cmake +``` \ No newline at end of file diff --git a/rust/ballista/docs/images/query-execution.png b/rust/ballista/docs/images/query-execution.png new file mode 100644 index 0000000000000000000000000000000000000000..b35240282bcdf832aea82da0c8acbdea492967a9 GIT binary patch literal 11378 zcmeIY2T+q=`zDMcqO>R-(NLr#p(`bHklsNAB8U{}VCY>yx9UTzaaauR=yvpGtjVM?pI0wAV7xCnF2yBO`nM zf{g5l)b;!q8JV9L8QI!nGBUXwGBOtLf>vWN>4KM$o|(qk+1bv{&fMJG@$qqAUmqL} zCtblW(O=_DP;@jSBNJNJR#!0#oZ2Y7 z8uDJQv6+n}nz2W=SWVL-gm~t4@=YhF(XUHrisxts`=c}dDxc+y{*^1F*!-{fZDxzD zAw$`pW%3xixABq5+}Ba4SiUlLy8p~+`{@_z0t(rYA*%As;z^$+N?XfoOxoOa@u-tP zFnzO{AK`Ji)WfVY-mX_JJ$28DKzfFZ8bJc9z7hR(HD7HKGlekBnN08uhcQNNR$dq2 z%VJI-C+xAJ?6CBb)vwNqpG=M-ZQ5L;Xp;;rwVdTEjCDpCjI?YXA2i+gFnofQCS^kO z8iUB$=~>ib+K1!E2Dm3_(C}gkmHSOMb*ea>t7s?F`}=P2>Z&E?G)hOttKoSjKhTy6 ziTF%{>h!JwzC7FSzR4#Q*v@RBuTGJQ6PoSPdaA}GinH_;z{0pk1uqgj^W5Llu1-HG zM7TPA4xR}E0~6W+PC7B8TG>5?0K5zchlriO@I+9-5PR+PEn2!QDFZ#B zUa^7CQ`!K_z7GW`A)|BZVP2os9Msfy>bOZl-{h#~f4uY7`&w)L?xAHdGoB1^ZAbwL zt#-J=LVIfdVa})j4dnD_j!)`|2o_cy&YjI=wrH* z;Zsgv07X>L4Ozg2=`)>=akdRB^Vd_4FLr~|oYROaS^6$;C!50;fivt^%B@MVm!e$K zG=SW|02)#wvr^^{HU7ixYkEmJKHeeWT>rX0e<>hqmmxVGbu03*l|y^5!(X8p!1AKe z1T|p6E+lpL%6{~r(fB56F}u*Yz2|pN#~)jysm^f>TTAhYPdm+W2W&Hcg0H?9&+6*E zU0av_tA%!aDXlD_;PuFE4-+ElQ1Y)3FlWgPTq)DC;yPN)j?Ox{vAgoAtqi}q)*6-F z-Oe&f-VmCMI(YN@%S@-Zp+SS0Ne?ioAKed7OAIH6+G1eU-)B0r{1(1_)s6#m zTl7q4%zap6bh#=0d6WCxviNY~hta`68p@Q&8QU@hd`^2uRCDRo!>ab=h8-h>cnq3p z;A5&P(IPg8B5S)sbLv+dL}!F`z@$2YVq8u>`WML{R?$V3WTbRi`n}r2d24;YymF}K zg58P|%uVm>w54{BjNw@G*R@~bO913hp$@%HaxJI$Y!qJpxRdGpJ2Je4s< zb(uf5av=B1^#Igd*sN%_&t0hWwm{ZYj5ZEE)Fb@hb{QY7se(F>j_#=*Yc=z>R<4=i z%N`dhX?Nkp)a~k$aw%w5C=c_%lNQc~RDF6U+8EgSKsOEHiPwt=uLI^(#21BNW}6Wd zCADj zgqZ+>*q4v()P-Tp?MmIhhoT@UedH$$hEbMI{W5z-&Fr*zvWlN-VuLkP$1bt&XPp5+ zfH(IhCE??pw86D4Hm~Fj<}(Yh`&K=0K3oiCFM@XDK>J;`xkUIV>e%)br~v#a6?)aF zCIU&0bSFoC{`lrGUAW+MGO2W${t?c>mwIw$%j3z;5+b@G<6R2;!W+mFlN_1!-xWn0 zfZY~e{xYTY?VJ0H&qjutFl~>EtWr}lysl^70vias_!5`?rS-zdMswBQzACC(_ za-@bV>zEue3^A-f+nep6lhRA5meP4^y8CkI9^YE=zUFmgk14z>JMiENPVNCT9aFp;@b#Z850t zPi0#`4$JVi{Mx)dphATTF7x zofC+~L1Iu{N73xgY#pO@vs|DEbJj`D-NKpsyb}8qA!YwqC4J|UKEt{#V+?c)VYE5R zgaK?M76qtg>cgRXR*=nZ3Q-K7eA~~}G$_V>iqeTf0Ukn%4p)z}8ZuTv3Y0VVZ z%Ytc~&vDoLdYESPYOr-PM|Y$sM>{#Pl66yWTon{?ZgR;dN5!A%(L4&$-EG12{sH^d z0^kpA*qkWrL@)xvVU~^%>sD5KbGD4fVnr&-Sy6oW}XT&3%NFCi?`mSKGlEm6(Ma9mv88NvM zQ~K(|nUDZL7ilTSQNXJmu<}syQP60N`93{>BNC#y{`Jk%`K3)8JIfj^Tmv5Xy%5-{j-X3}v>Xu8lii!nyCz1)iCUcH&j$PyT~ zer2pJWOcp}*=A9&-{$MQ`H87dyFDPFPF&-f&(;KoJS}O8S^r5Ph287>E#&K)e9Zi_ z#}e9~ywaz`SFI!PEhQ_^n`l4Nz?b_YM8|^0E{ga#1x~DoAki;3y%ir#a~%+(j9_Qb zz(m+QwyvC3+`EB)M~Q3oxdzDhh}fl_qQfp;X25$>qy1Jmfp@2gYwV4;5GY!{g?VP! z`ezled-LhUJ!g4HD!X1^uLvi$FgozN`|(?^KXS>Pki4_&dinT=B3S+~D0ucZs#gSg zcSN+V&TC6}_Sf>6o6ksh0b6Qn_J)7Ubc2IFirBdk@h?F^%$29J}Ztxp99%N z`Ayjz!UEo4=ji?IBRTCGXg{54+puVst^MEMHVe;!-e89#_N=^M>zDE0uiqn%xxPI+ ztdN4OXEt{oT^9YZm3_)FjIG<1xdZ8NZ}Ks03<;hKkMP{N1q&e6W&Fd{3*-R0Vn;I~ zayqQ8N$R9!Q$LlnsLT17x7g#=^sJSyl`GrB^T0-I=lFpyW_`erbT`f7vHSPV zB1ZiWRdr?oHQfzoy%vAXGID7rN=@!wBWQ*WOo9fD&)17n9 z`=$u5BMHG*ecFoG0NFThtJBPNzhkZl^RA`YB=v3GJM9PZn{p(d>$#7JN8?<9<)X$f z?z-K|U?EO7r`OAgiX4vIihI0!s)hgrhoo~Ix8cCXwZ|!OYEYMGx}~XK@%Z9l{gtAc zILN&O)X_#h!vB#{m_IV;q#czp;N45QjTzS_$6T za_H8yn>!6g`@Ibe4GqV9@Wk$e*Hwz^Ky+=Jy(FmiS4&)C8)GMp$i@Y4q$2O;s~`-4 zt8Nbrmq)m8tS(Pa5j_q#tHu=Oyl3rM|xRyto~qsXzmMQMZ=1+iumR@|~V zT933>B$f67)!AbkyQ?ERx@-)p51z8we@S)hp{|~@>Iq@h#9-&p1fgtOnfOE&1dzY-lHWnU%V>Ry;hURZcoFF6b^P~Q&?Daqj};`FC5@}R95j;{_RFxr^@uaoaA4r z?;)edFPv}oZtR;q6V=J?ap&9w3g1de6{lW&XJ}=BOGFWcJ2z#nLuX}w^xJ0&d*aga zd8MCSbD~{cnY&yMu9Ms<9>mmLU#*Zz7Hp-(Z9A81@B+w0cW2?-u`hY33+eF(T=vcrxRU{GLa zu9z4ZuU$JE)0Q>-E`n}=SJ?;z@r4d7e%0X9wQxVy1S}UWk^*6mP`-rm@qZzbLC5>F z7Ca-FZKnsB?9 zkeR!dX|NLJ1E1OfI1)g>^NlQ=5iO{3%44GnJrd4#p>Y|1!i^PW|KhfIwuXFfP{jzH z<<7~5`9{NnBm4?|^_4@o^3Y6}CUV+UpIpCf>t#1m;P^Z<^_fumrqB=;<7-S{bsEPi zoD?s<`=P28Ba@zq-n}o-QdnH9#>^l*7@2uJ;kd?#Pl+4tm-N_%4AuDP624ScW<;r7 zKNaIh{CMB0(>|9243zrp{LAtnW@rI_fDpq%9Fp&7NKk$^tG?UuN{3HyZ+SZ*W9cRW zbeV#GA}PxzzE9~h)`3KPHN-~vH|5JsR@th=+{|uJI;xll2v%M-wzTl8nuAwWm)Q%o zjR_<*{x(6KXy@{meWbH^q=(IjLa@AB) zIA@*fb%Eeyw(e?BZH%Hql`j<=0tZzsPy6~rN`XF!Zb?VOvai>R6MN`jl~F0LP*n*d zzr_5@2*fsq>3dK;7B?H|-;SnQYpdooss55L3lsJXG|IKL8z5d-n7;#C4}~fSGHDu{ zr9$y`J0+0P%YYm5jth9o`_Co-+>SrQhm(q-82nN9OT~qGc9OVG7`{o2?(7o&wt*x1 zJ2XlU0K7fx(==-`S*U1m6pomfom3ey-%np#M_vbbMM7ANOz+jZfoi^@Oe`_sjaytyIw4EsRfFILt0QGfD_0CL|jCIIBVkb z?kqmDW*_`&+)dtAo(a#?Of-MwGI!&FQQ9F=Eu(qX(bK%JBIbmsQt>&#MB~J=bnf!_ zxQl55WSlSQ)v|39{tDik3GH|F*@M;m*LQaE3+5Q#3sj>uk6zT<*e=!6?9qR#%37>= zhuCp6wad0b3|E;TZIrucW$cVWJI{^7A6kzHERx8-F(($To>8?8^_prwaTEmd5eH5g zdgEF=avVAbnl1tu0F;rCfRtmrccHAunRj<%Yu%abCaTwOx$(4@Vr;FGNTN_a(<25y zZj(q9s4F_i<60_m<0=1E8i{9R&vh4QO!388@NfeO&mjSfLzd`a5CEa24qm>1+!V-Z z&>H3n6vtEJb4dK`&EwD9x5_^xHmZXWq~th1Ei~Y=uuKRXx(qF38lo&6;ND-fn<&hu zx`^D&;|@>)KA1&TaRLOgCmzne>agQ#(vA-tqJYjCi#<&<7*4#NDoplQ%P*Yk8Xh9C zAPzwLX_8E;9=75HV#)s{2+WB&S8zTEKUhn5JvB%fAy2j3qyk=UE-q}>yh~z&;gp1_ z|GuB?cjHkrdkq=;=*u2)gIkYn*!%1~k0&u94(H$%AXZ1aAl`EN8>jG|ghJc$IoQki zoau1?ZQ$hV4evs>AsLqm4$9x6cn~n<0un^$RLOC_f~e>e5o?S0CRuGM*4r+|(kdiu zcgzk>jLT3)P)Avo_x~Z#I0{1}o{Rq74W44UruZrHT6W zvkPa5apnP@LTK`VES9)74%~Y-z}w9kGZw0ivEzh~KPZaV?cvV;GWbwQOc!nH0e3kDQo?^jI6K!fdHr!_% z3jIX*ew-)&GR|HZk=qUCGqSW$*U#UoOTK+-82o)nzxn1WUA+KmoH94!%PeE9&Y56C z=x}}Qj5ZkZ7X^6PN1YO#=Ma7FT*hlTkY{QhMGcP+VMm+1Q=oczJ+4Oo z(`!Q!0#W~4m$V}c8}m}$Txy_+wYjQAJk+z(#71^nR=FnUG4oxxi78k7>b{MQj%JIw31<#;2WybnbgL=IcY_Z z#l63TYtBvZXn(`calNrhlxsXMHrjIy_`09F6MHG~bEJ)$nqWp)^k{N|)o)ZaUBa8Dz4I zYUJ4?#%aj)N0?Tx_Da;^d`?wpkVMi=U&iBdvmHNrWbAFR$%Y0#`l}|25~I;5ok6Bm z!#74u!2&FUjIm`YJ0%k`cierwzGd@(K1{FkNi}|%ecq@UVZfIZ1&N?J_hB-bk9yZP zQ240qSxs2RBI3oAm>CpiK6s=a04h`U@zgaQK3u3V9DekWo+r6|z&cDK$;TA7ZmD63 z#q`s~1=ATrh4;h(Lw1nCH^WBB(8AD!Pp5sobg=ci-FJ>{&7EepGHa|fpvwKXlBlYQ z5YxUl&9OqVOd7OI!t3x8gvR_Ftb&)_r(+Bf;T-!ps$dKg2H+dnw%^RQ@ z{jr@sp&4`Vou4@nguV2Gh?I?QdkR3BjcwVdw0<2v)xdaU|0{rBM4Mqo(DMdnn7^!(u* zSFrE_S=;}oim+?O_gCPlrM)r2|6$jr;rey>EtH1Q@sf6?$z%kaV<2K9@jo0o9t$oz z-8vb?447nP1W-bEW%ovX)ph9qlrx|9NJswL)qNBc-c?($r>lfZ2fTT;HYedvJt?wA z_8;CCK@zH`DDsVsOQJXpvIfXSEULzz43!KgRV5(f1`@e5KzB@yYv0G3F$*wNhDVMq z#pv*m*TwHFKW!}b2;Wt)Gcv0(s>6eG6cJEv<8JH98OD@xLxT0-_>RxE+#;tBVwdO!C-SHfN54qd1npqXU zn*9n*0IRTOofv$A($dPh8rb+KP1MF~u`xxY=|+6Y6%Jxq>&6xAHPixdQ099}Yw>~c z6*<)zZ^yWQ@k=potW@@WkTC^cl!Eqmi$a&~tMor{v3z>@d$S^J&~hg8EeG@c`=H!b zF=*2)r4nx7O=W(pXr=bp5L%jNDqC23{mn--74r85_5+w1wG^|1vm6W6N!KQD|1Lau@z1thQs)6I;1D;pKK@)#;#Aj zlaobZZm5E=&mm7?&6s(0G!nXT1RplBZh&|O9{TL>R$&p@gDuG?4d1-Jy=9pavFw|u zAjVnLD$Xf#Vt@Zj!^u_X)i(pZtJO(M!ND1dF50%>?KfMq4zD}#yNg*$ z$AL%O!Mxh7xRKR@#irs&YJ%_?YY6js)jPR5yW8F0SEm~nGxI2H%(dzC2e|f^AE9yb z>+zUmM0*f?LjXHw_3B5^>6oh~M|+~;$m$2WB`QKx?Azi;WTZNhs$};8Qk>W*f9${d z;<9L+OG}2$ufn3;Hqi$wh1rVOIA~ZJ&5h#2tPn&)rL7TXegsr|iQ4oar=RMEg0vd{HhzbAWjz2j6HF&Jfl`K)qdh)oj~NB;kr-P_54XF4r>3z;dAaB83 zK%wYs-Lk2gj*$2wL2U42N5u0rA{!`fYUgP@mfZ#@n!fhU%b?5CRmkGqO7{EN6pNn2 zFzn&QR`uY=4}M||4T8HeH~ib_KX0{tX8FP!ei#Ecglf|=`;jA!l$HMIf$YQ=H)jpz zPde8_FWf?@AvZ6ttS^(Al+a1{kZImO9nVupZ<(4qHs}v*lu?n8yVF1XuZD`gUQ`C< zdw8{7>SX!yj5B+5(vG~QAUmvp!XCED+WvoQrxJODMhtEgr>%fFvlR;J z^4%!(MpOO-rkQtT3c}x;rr)ud)0IfG9H z_EhECK#x8NHN87M5lluKmJ?q$nV>@1?I5awh-ss)*HP>j|C!;Z8A7HmF%R&{Of9jk zI9)R`NqLaT+>epwVwTw-HyaBLP}~C{>&9Hn?K9&y=_cDgXO8c)gs!GncMjOG;@-y* z{CsMkGa$FijVYwzFNqIRDodi`c}rBy+Z$PNiE#R>=6)JkW>pJ2e1MS3rBM__#Q3Wb z)Asd;Z(FX}Jp6R}55K@t$)vq~T=zYmG4g=ma~X^AjqsBBZ0Q{5t?*)&h&*eksOvRaY;)H0smp*uEHFe>)L16%h+@ zGj|~i{oqfqehDiT(A=~0aC+ZlF24_g52zu{&IS_RZ0>juX*#%Axgp1m>cfz}XWy>j zS{eSX+s&m>O>29uVF7W_{?d(YUJ1hWlWYjJM}=OvGUE8-VJq^%M`HvmFNdj;L6Aq( z-2CRR`q?_@s&?{+(J>C!%A8kkCAj;8PgOa5F!Z?XTclcRb%DPjMp_#DIfGbUe)p=x zX|vp@PmH?XEaS;k7^2dkRlO-U(ZX!07IH*-I7R+#{dQS=MjwiMWZ+;d-&ZYuM#OJ>$@;>E| z!6MQ(8>;`5?*hrC8%23dU}@B#JWg)vpYpu|LE2zrCor-G->yNCu>=a=ghqY33bW5y zZdg%|S5tXQ(B8q~wmQQO{|-o}2d=m&yuqg9izkK~p}4Em?z(%pcLCY@4_MF(rmmQE z-ir>ZhZuT|@{6n{X@>7~Os^UKNa%EFTb_6V>~GNorTMc^8J@KIK19z>+^c;sy)`_RXmign|5k^R0s)n8F_i*UsC8T|#46SL z?{vTYMJPyFH$sSs{&L!9ICT3rzV?Qyr}GlHle~Q{bo+{DK@0P zXnUg^cXrL3cU==WVEfDO=L`*3pr>{hx|bLaX^;=5H}?H`*V+TzY>tpri(jlovAXxX zXYgPCgd&;!nrvPDEFEw4H7ooK{!L3=rQajl5OZgOkY}TDi)~Fo)B^k`1MX#|FqUs~ z4f#V|*zydMv`$#6j`*DpqyB(WM-X3P;V6i%zs5+9Xz^X^DTS>anzlP(2}+HRIwQP> z~kA_#V`ua;$dnYD;HI-31QG}o3%Jy|9JvGU~{|tIbho$SfzN=(MQ=EL1 zM84>+EjqaP(Hw6CB0C|2Uhuxn$wGGT;{Vo0&mQ47zAwbP<@*Cl=!O5cKQQ%HASwmA z;Sk+dc63Q|^ZIH$^mVZFb(FJ*I+6}#5@HhKB4QFE65?j!GIHW~vzMosKtzn>!P< +# Integration Testing + +Ballista has a [benchmark crate](https://github.com/ballista-compute/ballista/tree/main/rust/benchmarks/tpch) which is +derived from TPC-H and this is currently the main form of integration testing. + +The following command can be used to run the integration tests. + +```bash +./dev/integration-tests.sh +``` + +Please refer to the +[benchmark documentation](https://github.com/ballista-compute/ballista/blob/main/rust/benchmarks/tpch/README.md) +for more information. diff --git a/rust/ballista/docs/release-process.md b/rust/ballista/docs/release-process.md new file mode 100644 index 0000000000000..c6c45c3cf1770 --- /dev/null +++ b/rust/ballista/docs/release-process.md @@ -0,0 +1,68 @@ + +# Release Process + +These instructions are for project maintainers wishing to create public releases of Ballista. + +- Create a `release-0.4` branch or merge latest from `main` into an existing `release-0.4` branch. +- Update version numbers using `./dev/bump-version.sh` +- Run integration tests with `./dev/integration-tests.sh` +- Push changes +- Create `v0.4.x` release tag from the `release-0.4` branch +- Publish Docker images +- Publish crate if possible (if we're using a published version of Arrow) + +## Publishing Java artifacts to Maven Central + +The JVM artifacts are published to Maven central by uploading to sonatype. You will need to set the environment +variables `SONATYPE_USERNAME` and `SONATYPE_PASSWORD` to the correct values for your account and you will also need +verified GPG keys available for signing the artifacts (instructions tbd). + +Run the follow commands to publish the artifacts to a sonatype staging repository. + +```bash +./dev/publish-jvm.sh +``` + +## Publishing Rust Artifacts + +Run the following script to publish the Rust crate to crates.io. + +``` +./dev/publish-rust.sh +``` + +## Publishing Docker Images + +Run the following script to publish the executor Docker images to Docker Hub. + +``` +./dev/publish-docker-images.sh +``` + +## GPG Notes + +Refer to [this article](https://help.github.com/en/github/authenticating-to-github/generating-a-new-gpg-key) for +instructions on setting up GPG keys. Some useful commands are: + +```bash +gpg --full-generate-key +gpg --export-secret-keys > ~/.gnupg/secring.gpg +gpg --key-server keys.openpgp.org --send-keys KEYID +``` \ No newline at end of file diff --git a/rust/ballista/docs/rust-docker.md b/rust/ballista/docs/rust-docker.md new file mode 100644 index 0000000000000..0b94a1499a0c3 --- /dev/null +++ b/rust/ballista/docs/rust-docker.md @@ -0,0 +1,66 @@ + +### How to build rust's docker image + +To build the docker image in development, use + +``` +docker build -f docker/rust.dockerfile -t ballistacompute/ballista-rust:latest . +``` + +This uses a multi-stage build, on which the build stage is called `builder`. +Our github has this target cached, that we use to speed-up the build time: + +``` +export BUILDER_IMAGE=docker.pkg.github.com/ballista-compute/ballista/ballista-rust-builder:main + +docker login docker.pkg.github.com -u ... -p ... # a personal access token to read from the read:packages +docker pull $BUILDER_IMAGE + +docker build --cache-from $BUILDER_IMAGE -f docker/rust.dockerfile -t ballista:latest . +``` + +will build the image by re-using a cached image. + +### Docker images for development + +This project often requires testing on kubernetes. For this reason, we have a github workflow to push images to +github's registry, both from this repo and its forks. + +The basic principle is that every push to a git reference builds and publishes a docker image. +Specifically, given a branch or tag `${REF}`, + +* `docker.pkg.github.com/ballista-compute/ballista/ballista-rust:${REF}` is the latest image from $REF +* `docker.pkg.github.com/${USER}/ballista/ballista-rust:${REF}` is the latest image from $REF on your fork + +To pull them from a kubernetes cluster or your computer, you need to have a personal access token with scope `read:packages`, +and login to the registry `docker.pkg.github.com`. + +The builder image - the large image with all the cargo caches - is available on the same registry as described above, and is also +available in all forks and for all references. + +Please refer to the [rust workflow](.github/workflows/rust.yaml) and [rust dockerfile](docker/rust.dockerfile) for details on how we build and publish these images. + +### Get the binary + +If you do not aim to run this in docker but any linux-based machine, you can get the latest binary from a docker image on the registry: the binary is statically linked and thus runs on any linux-based machine. You can get it using + +``` +id=$(docker create $BUILDER_IMAGE) && docker cp $id:/executor executor && docker rm -v $id +``` diff --git a/rust/ballista/docs/user-guide/.gitignore b/rust/ballista/docs/user-guide/.gitignore new file mode 100644 index 0000000000000..e662f99e3281a --- /dev/null +++ b/rust/ballista/docs/user-guide/.gitignore @@ -0,0 +1,2 @@ +ballista-book.tgz +book \ No newline at end of file diff --git a/rust/ballista/docs/user-guide/README.md b/rust/ballista/docs/user-guide/README.md new file mode 100644 index 0000000000000..9ee3e90fcf6dd --- /dev/null +++ b/rust/ballista/docs/user-guide/README.md @@ -0,0 +1,36 @@ + +# Ballista User Guide Source + +This directory contains the sources for the user guide that is published at https://ballistacompute.org/docs/. + +## Generate HTML + +```bash +cargo install mdbook +mdbook build +``` + +## Deploy User Guide to Web Site + +Requires ssh certificate to be available. + +```bash +./deploy.sh +``` \ No newline at end of file diff --git a/rust/ballista/docs/user-guide/book.toml b/rust/ballista/docs/user-guide/book.toml new file mode 100644 index 0000000000000..cf1653d74554d --- /dev/null +++ b/rust/ballista/docs/user-guide/book.toml @@ -0,0 +1,23 @@ +# 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. + +[book] +authors = ["Andy Grove"] +language = "en" +multilingual = false +src = "src" +title = "Ballista User Guide" diff --git a/rust/ballista/docs/user-guide/src/SUMMARY.md b/rust/ballista/docs/user-guide/src/SUMMARY.md new file mode 100644 index 0000000000000..c8fc2c8bd6a67 --- /dev/null +++ b/rust/ballista/docs/user-guide/src/SUMMARY.md @@ -0,0 +1,30 @@ + +# Summary + +- [Introduction](introduction.md) +- [Create a Ballista Cluster](deployment.md) + - [Docker](standalone.md) + - [Docker Compose](docker-compose.md) + - [Kubernetes](kubernetes.md) + - [Ballista Configuration](configuration.md) +- [Clients](clients.md) + - [Rust](client-rust.md) + - [Python](client-python.md) +- [Frequently Asked Questions](faq.md) \ No newline at end of file diff --git a/rust/ballista/docs/user-guide/src/client-rust.md b/rust/ballista/docs/user-guide/src/client-rust.md new file mode 100644 index 0000000000000..048c10fc9263d --- /dev/null +++ b/rust/ballista/docs/user-guide/src/client-rust.md @@ -0,0 +1,22 @@ + +## Ballista Rust Client + +The Rust client supports a `DataFrame` API as well as SQL. See the +[TPC-H Benchmark Client](https://github.com/ballista-compute/ballista/tree/main/rust/benchmarks/tpch) for an example. \ No newline at end of file diff --git a/rust/ballista/docs/user-guide/src/clients.md b/rust/ballista/docs/user-guide/src/clients.md new file mode 100644 index 0000000000000..1e223dd8eb05d --- /dev/null +++ b/rust/ballista/docs/user-guide/src/clients.md @@ -0,0 +1,22 @@ + +## Clients + +- [Rust](client-rust.md) +- [Python](client-python.md) diff --git a/rust/ballista/docs/user-guide/src/configuration.md b/rust/ballista/docs/user-guide/src/configuration.md new file mode 100644 index 0000000000000..52b05b0e91679 --- /dev/null +++ b/rust/ballista/docs/user-guide/src/configuration.md @@ -0,0 +1,32 @@ + +# Configuration +The rust executor and scheduler can be configured using toml files, environment variables and command line arguments. The specification for config options can be found in `rust/ballista/src/bin/[executor|scheduler]_config_spec.toml`. + +Those files fully define Ballista's configuration. If there is a discrepancy between this documentation and the files, assume those files are correct. + +To get a list of command line arguments, run the binary with `--help` + +There is an example config file at `ballista/rust/ballista/examples/example_executor_config.toml` + +The order of precedence for arguments is: default config file < environment variables < specified config file < command line arguments. + +The executor and scheduler will look for the default config file at `/etc/ballista/[executor|scheduler].toml` To specify a config file use the `--config-file` argument. + +Environment variables are prefixed by `BALLISTA_EXECUTOR` or `BALLISTA_SCHEDULER` for the executor and scheduler respectively. Hyphens in command line arguments become underscores. For example, the `--scheduler-host` argument for the executor becomes `BALLISTA_EXECUTOR_SCHEDULER_HOST` \ No newline at end of file diff --git a/rust/ballista/docs/user-guide/src/deployment.md b/rust/ballista/docs/user-guide/src/deployment.md new file mode 100644 index 0000000000000..2432f2bebb1a5 --- /dev/null +++ b/rust/ballista/docs/user-guide/src/deployment.md @@ -0,0 +1,26 @@ + +# Deployment + +Ballista is packaged as Docker images. Refer to the following guides to create a Ballista cluster: + +- [Create a cluster using Docker](standalone.md) +- [Create a cluster using Docker Compose](docker-compose.md) +- [Create a cluster using Kubernetes](kubernetes.md) + diff --git a/rust/ballista/docs/user-guide/src/docker-compose.md b/rust/ballista/docs/user-guide/src/docker-compose.md new file mode 100644 index 0000000000000..2548e57e5a759 --- /dev/null +++ b/rust/ballista/docs/user-guide/src/docker-compose.md @@ -0,0 +1,55 @@ + + +# Installing Ballista with Docker Compose + +Docker Compose is a convenient way to launch a cluister when testing locally. The following Docker Compose example +demonstrates how to start a cluster using a single process that acts as both a scheduler and an executor, with a data +volume mounted into the container so that Ballista can access the host file system. + +```yaml +version: '2.0' +services: + etcd: + image: quay.io/coreos/etcd:v3.4.9 + command: "etcd -advertise-client-urls http://etcd:2379 -listen-client-urls http://0.0.0.0:2379" + ports: + - "2379:2379" + ballista-executor: + image: ballistacompute/ballista-rust:0.4.2-SNAPSHOT + command: "/executor --bind-host 0.0.0.0 --port 50051 --local" + environment: + - RUST_LOG=info + ports: + - "50050:50050" + - "50051:50051" + volumes: + - ./data:/data + + +``` + +With the above content saved to a `docker-compose.yaml` file, the following command can be used to start the single +node cluster. + +```bash +docker-compose up +``` + +The scheduler listens on port 50050 and this is the port that clients will need to connect to. diff --git a/rust/ballista/docs/user-guide/src/faq.md b/rust/ballista/docs/user-guide/src/faq.md new file mode 100644 index 0000000000000..b73a376988b51 --- /dev/null +++ b/rust/ballista/docs/user-guide/src/faq.md @@ -0,0 +1,31 @@ + +# Frequently Asked Questions + +## What is the relationship between Apache Arrow, DataFusion, and Ballista? + +Apache Arrow is a library which provides a standardized memory representation for columnar data. It also provides +"kernels" for performing common operations on this data. + +DataFusion is a library for executing queries in-process using the Apache Arrow memory +model and computational kernels. It is designed to run within a single process, using threads +for parallel query execution. + +Ballista is a distributed compute platform design to leverage DataFusion and other query +execution libraries. \ No newline at end of file diff --git a/rust/ballista/docs/user-guide/src/img/ballista-architecture.png b/rust/ballista/docs/user-guide/src/img/ballista-architecture.png new file mode 100644 index 0000000000000000000000000000000000000000..2f78f299c20bac3f3126a1875596fc3b63563185 GIT binary patch literal 21225 zcmd4(bySqm_dkk*%+Mm;odNN-Su-VmYiqKv(G-~?Ah_!dlRptrTh?=8W#->?V+lQf*u+g2#SXG z02>TFSpUq{2*v( zyHGSVaReF~#f$tdT}gltxsHZ`;@#aHFz2*wq&2neZoqeAW5ZG3Nm}5UkEI{sLn1R( z^RnoQrsS6V@WSsk_T}N~({HS=`n~zsArTHQ(*rUoiE-0k$nCbcm?}IjhLzSP)W>@! zMm`sBOVB&%@)*oCRS=?Qq-NBV*1Ed7nx3AHjEtnDr2PKESk%q>svz+<7_ZJ*-sP z@&Y0~45!~F&KEZPsOT_!rnp<*-;tV~FvzAeaxxFkzn1YCSG_x$p z$tP#}=x)E-Ha1^a)gWu~Ai85l(ay_GBgpepXaatxm4Qy5NZBX#6lp|-v$0NcN(qb@*6qK@Hwab(+>#Ec$ zNfQrr)2qo5bcGT!(KE=1tL4N$d*#orqe995#H6E4G5IAn(IcbgA|5M)^78W4)zy)a zk*{CBW@Kda^z>+JYy0^4l$4Zka&o?U^-4-gs;Q~z`Sa&wWMs1W6heSlpg&hvRzQmb z)2RRhoEIvlK4@r^dG|l)s(w)!z(XuwRZT^#jR%;INca!(xH{3$w7XRmWDNqBe)Yyj zkbayW{)K$}_%)&Mcp?*yaf9|+Fqne$DfTNK6%~gTM-}X$rv6XqVFj#mGDf;Of7XV^ znd6x6RztK5JzE{va|(BIuKO4SXY?&C)2XJdKUMEG1n+VJ%l?1#Pew#8k!hx1a)g)s zbU2IPPmundXP?`u-Q^xS9_zyhw){MbZx_^o5wR9k%e_HDhChaQHOyhA-Toru-+OP;!=lC|uy4oyi7$;9s z_((402Cx(sJu%9qK7*prs=mF@>fPmCP@$DYJ*d^YUe|p?&9z0}aqq(?TM`Pm&@04VP5B-l|st3pHmvk}^ zb%I75hdH0OyNba^A6Y&%gvu&I_KK`;_=0bDDvK(&jCy*k+OuYSKJ}N4(PFi#!+B|@ zyNfc7_pG01BVa)uH@L0W(bo$xkc{&=(BiOp5!6xs8b<2u;ul@s(PxYR9kZmR`mWzu&IHsjND zLjE~u3mq|=n^n((vc{T?dmg^Mmb`hAUC-_{uFm(76zmORAJfXUrpw07D%*?a8mn>1pSoU~`jcXUwe`k&WOV41=@#)ZZVoprYM zVLa_Tm}&JigMXw+q;+H4I4#U{QuULXKE&0q{bTINj}tX>NUd-7Z#BaPy-JF0Ya;(< z(D6{7XR6{f%w{F{IH?SzZ>FR3K2lBhoj$}U8_tICuJIp362hz4_!`Z6x%j!*I1u3Q z6@`8bY=_ZRPUI8o+dAK-Kk;{xjw*cSuqm_EwTPWuzOVL7tDkVIo4!$}8O2Iooy?p; z+IY8yDV(EsH&btaQv9X;yD;_`h9mZs{4wR;wVDg5>S^g0PGX8|@(c;EzKQ`Q6GGU) za!logyrM%8G0gPumgb4XOK27?Ju$qr$nVCd_V)B!%?o6mp-pUpX9=#-DPb(0QvO>J zIY_PH`Gnd{@UO&iLv}JQM2GyRcE!W_uXn1d!L@mn>x(J&$WgfJF&(^-NHmk7mWc-zur z#A3v!(z8y6%%DhJ^y4m0STU5faKfulyEHZkGJQvfG%)QlX@Pm2%cCXMk)zClwR*3! zC(|{|CDpl&>Vs3~pV3)DxbMhPnVXZ0BzrHTv5`%+Ms;Srg~G>mYFSw`OD~Xi?225U zwbx?Rrtm}98p~wFeG{Dv;iqfar zUY{#f$rMO1kTG<{tw}VYRm^l9JFZQ{or?}sntT3kCBU*8U(vjvr`*7@xk1G;*b%1! zPTYOWXe=JYIperO%(Mq;4UQI!37qP*{jovD{a4v@^yk=Ck+pmObjmI?y3IAZH)Jkh zgL^*LzU$jkEpfRa-?RSfixICfID!7wI7CnEYiW%I^J`q^y0`*;6Fcj-P}TS0Gf{QB9KEh`NZNa9^jj);mk%xmjAEo_vk z#T4SYAYFiI)L=!&_b>`-@%L#g)H}wF9M19bzXQzwADO~*B@ux<;>K$aCuH~UUWU@$ zE2fZ@6jQ$Ce>OIKl0@qmdBy+a_`t!(btaZUb>dM0+0*GP%3mHhi>+)9y>BC|{&6xL z_#vz(`r_3`BHMoZ7@iYkCC#(26lrftOql8ogL%cn2Y?B$mRw!Y*UsBv3D+4-3lBAB zlh6BJ`3a8iGNa5;4c)}hTby*r|;&Fu9TmPc%~7pYchSlY#%^`okkcoXtj?j~GUZxE z#Fzg?<~7fdWn58h()lwRXSdbujb-g;7Ya!#{oVG7L8{+2&!1h`B?jG$T@{TitLM#W z-Ha$Q>?Q_3jW}CXvOK%sD5~wHnV>dbugOg^7x2womiUiTOL6bv{_i@q16Qy^@xMLQ zclX)?^AjHC6DdZ-H*^?!AqH%e4)E5*{=Xiw@W9Y_00yjB=wRq84`Ard7%e??XZK6eEAj~3Ghn!TBHQm!FA*d2$XJnKS^P3xSg7VP$T(%N+hZCzo|sp zkgox=2j4J}>$HG#OVbQ)^~v6itd4|w1Mxvw5(2iU*?l%Ue^6ga_up5E^dS;XwZSW~ zS}Q=%fbPsv0o~m_@!@d?&I#|o@3eY$1h1%fGa0QPsl@$%hyps|BX9Rvcr!k1^xu>s zt4_tUX(!@+J1+K*ByL64nd+YwTWh-iZg`MXV?ufvy_JMw$&{mFLG{GP?lQ%r`t?qK zY>?`30O2@&NWjk44I$j``Fscs_K@vobkB5My{%t&Yp9yYrbhbNtG zk_B#S*c5Ekmk~hlkwKa>=)OvL^KZfVo1zadK+vhvlhAzn%DLv8exj>b(ApBLs z#dr54s*|kSbl>wP5Taxamt?W_!muRRJ|0yEMpvbSOY|RyyBUmT*Gw(GPttSLN15Qj z4k`a_&O!jT3aV-=FQA+LxI`H3yQoM6vk7=R+p#^cbaSiZQnU3_Ja17Y+#FK>ix?^7 z735Pn%ZTeMBoA3c2a$M6Q`!c0WicWYz&=fprLW1)WN6xI11siu1;8^aRZ!Kx4zwBI?j!@tlZX68^txbqD+O z(O_Ob(i=>OiK;xvlGQ{L4#g6g!7L8WsT^d;zc+m*L~z{|Bo-DUVjmqAKZDg8ao2M3 z5=v^xoJ8&d)mL#;%Wgp*mr;t(S6qc1f**wML^;oi=-xo4v3`2GOwl{@gGk`2sY(G_O)m>!j2A zn{i8FLSW?^=Tv;_V+#l!SJEzFtnWprWg4%aZu+N%5x?9(h7MJ|#7FKWm_?1UE;0~P z1vez&sms(W+_l<90^038DcWJ2Fl7Ugu(#THn=d@APlAWngx4M-%iOzS9xP+%TI$e; zMc9O!XtL{Ce;MxLpbV>4Pv6xqMiUky8jrDkEfZ#<+k0#jiCOSexJA{>kWKfI1lt>N zH_<>G5P^T%WNDcay(BMqjeHL9}|WTQ&jc)L-DPlc8yQ}jn%`=C!*XR zNHw$resd-(5csu91rzw4pZIeB+O8ZJ?)v4l-pQ0q;wy)vRB6KT=-hwLpMXv@6xI=} zrNigRu)Eo=6>I?6$!jQn>5I@+-23q1(T#z|s_+2kr;b@ta&XRYR)&cpThQVA+5;f zWVqF-5s7_qG1cO=3MuBU1xf>Nec zReh4jwZBKoD}ND6&|i|I)`x`ouCGTHviQbX(>)9Q@tU@7y3+QDpmLkr6PG<~Mi{YV zqWn<{X`rtduBfYPqsq3Ct)K+4$n8*aLk8*J=BgcPqLVeNC8+Hyoiy7U80hHMn^#Wb zVzbY1tmrW{K!dld$Wf#8J5|3nc+?@*s*fgYj0SAVv-mU{X-fxEVLKJl8}vbbs46?` z&B4gdC#gHs3N+Cd7hpw$H9`#>L!Xi6n4ihhBhr*v3PHMlL4HKB+?jfODVk;Bxazgs zv{`Ak27?qixB~A6y70tAV<4#~! zp#=?K(WXZvVK!lLM!{^&VhVM{=8)L!FZ^DkICF|$1HQoJf_GkZC7dJpHFyBW_o$+f zPdf%@bKNFY|IukMpklq|e#&upmQ$DWrj5C35Rn>re<8AjM9(Q+D9%McK`^&gb(z%} zG!0F~iMLmo{qoa?1k;P3A9f5a;vfUz7a*T)(Pn(Dsl&p9pUOLd--~|^hY;+TgOgKt zo9ut*Z%fD;(asDg(_S6bmZtqhz;=p%E{9q4e_}$=i!UAD(H9TI{{1nqLW&CiDUNuH zQ}L>7ygeaF&uWa>q*B~Wjnn!@0k;AkRyI^qTm7b~NoKlSU_~!7VJ=SEXbg{=mC{=s z;vT70^u05yp|5zc^G(CzY0jQV%Upd12ay=iyoW#x4}Vk*uo2@>Bf4RigSZB)2{lKq)Tz)BK>A#wa3dO z@Nd#~%MN86)%w2voIg!Eu%4*#@+8fpfZ)T3&>SMA)DhqJx&1d}OG>~jtfg}6^C-Qa zmq$v_Bjl)l@wp)VuGpNT8|@lHrX=08-wF3=pkCXzfZadE7|o+E671I6pPAv=-hwBs9U z&kc`PY?~O7LgSLWHn78mt82@G+0}FktkY0@uaXqdHuLEVLf9zz_R9{OJNdwkwuQUa zsz=wCu8j(zSp52B)Fd6?pPb~Z2$@R01Sq9-_csug__#rDS8tW6xI6W%)cn!NvwAv# z?h4yP+pJ{v{)VZg&vTzudp<2t#Vvlt0Hmjm82@W@Y5bk(H!ODC0MxbRyM}XQU_vyu zlJ+|qO~|Rcz(=`GD0Ou!Kcj5QWWajrQeIX1gNe!dEUkCZT{C>b;?=2-ocGfR+2`}W z1F_%;J+kL;^KRGIehOxDdp_}$RYPz0_!s{T6aR>R7ebWEu@aE zd6@jRQDq@WiHGg;Bzj6=HL_5K7Kk@YnGsJ{LdSO>sYXK)9Z!Y_2~qRn4Q*I3RTW5* zOH1}PY-+iG$UE6R2Ku1hUsE5VXT2HG+RrgyGk#F3p@-MEUQY&4IvjfPS4%w(I{0w@aoVEP*Ke|<^H}mrlHnn> zrYSB@#ma2ku#t9P;C=H%{}IWaHl{A<2`l6Z2c~KP;Td|+tT`E38Z{*zbV?AfG5bN) z4B}qHRMcz#gzYmu6J?poqTDO!Wmmt&ODy6aI_IJ35$6i;8gu2kto8Dlsp$BdL8S~G zau6IGC7d#SsWwd3H2!>VopMDRn6vIq>Ir^8i%zFj>EihF^Ntat5E-l|(&~`DU{@Rx zY<9djLKKmob;-BOZvl+DC-u|4m1~y#Pk-xyyl0T9aRR5Ye}?iJfrw=x$zx?3w)hCi z%0y|q-kDg{TbycOQgZ2+_aL}<0fPKn-TSVM?F~86w$rK_GXhPx%~uWI&T~|5?%@z@ zR$M48b`N@-^2-K6wy(6OK*t&0sM=~GRBdtg z(nqyamz>_W#)XWCAW*4MiP2wF@;8yK0(5tx4zmss8jLyj>;3b+7b(QKmi{qfZi;5ig$QOMX*tfu-eg_QZmHwkZpA zme%NoQ>ycr`uC+)e11TbWjNN8cj>oY^{v8)M6R#qKH_f9MR;QKB`{H95u!q2&Pf4y z&*i>6M9mnmeC%VR0pbZ8m{|DrOMOVMZ^;#Q!H@TwRh)#ak8=TY)k?8h_T;s1MlyqF znbG5&Visj>T9%-wpT2e7#*KM{ewPF@sRtefX^X!ZEFq|vr{sV5>0*^H2uf;fUF!i{ zTpDR2Kihz}3Eiis#2)f_g*(K5nx4zq+F|Vclhw6cdrR9Nz+?&hV*RIg4YjqZs;vzY z$|@@XC#-1cFf%vXID^%-h*q?WJXn&b5Mdvx)E)E^su5Zc`Ei-k-CBy`N?cra*$4+| zhqDUX@yAv2QX%I{-M|o)7vFg7ZT91@Zl((DL(VxL=d-H0jIN23t&BD^zoJ}=3gg~6 z-4LaTCRuN6c5~3rhm4ddM1ai(PNFXK$L(}~lI zfXb%1$Td96pjy6ou{=d5VSX9#>H|gu90U0>gNRL0+*d9Jp6qp9oUN!|d7FKbsb)>9 zxEcnj{LqjtVP7{wt_azr?I9=%wYO7ONfjWtU#t_#I+3cVGz;b*V_yzl-ubDVLjR0V z&u#+YoXku&<%)&)80Ndb9+^IDz<}bfgm%p(lk^Tvv&9CdP^J=y2T7sdQ>lWUKn`t6 ztW0;TM3U*s8&!e>PfJ_l9%}#|=k;fFYMzUtu;Rqsc#qE@xYe^29Xg zL?knJ#DbEl1zf%8fGj4t0s?Bs^96(BQ69x$fw6!C!@Wd48w{!O8qSR11Y-d4XDZq_`!5aRIOeUu^}$nXMJJ*w1YrpzlbscnyFd zKs-N4gd!;EBCX%Y?g4|+*%=X?uKJa3tHM8c7!gAMJ}%6?xPE}N<0RC`eF2K%5`I;ul8E&0B4OapHf*Oo)0WTatfJO&Jb0dko;9hyn-^06A#h0}>7BfdEPZ z{7Q(@1Hn{*iQo318qZgL1GWGFobKr_@0%#U^`w~M6`5%GMW{glAPxW_s|17Ah`ii( zo5hxbw+ygRrvG7%;_9USfpFV+*4&&7L<~T%fE@&eLApW?k z6zf7MKU*+r+%=bo7w<$bkUK2(SX^8|%CLYiBndpoRHC6>+i%pK=jKXO7w6c2UJPuq z)R=lN?u0!;ZfXyJ9&N|%MN46)Se&0ff9CYdT@=g@ZCiv*@ko5d^Yxd7y#FZ!NhH(2 zXj|ww)D8R&s%Qs^s9hherAnJ=lAb^po=VM+GLhV`{AYSEyT<@*6`JtkBbf zP};;wBstv0G*+#~UgV)3>yIEtgd(=mrL%+~hbPMwlH&{l8YqFPs#xeXT3wl7E9C?= zMZ%pJ%Ct^3euc+iP&4G*2Xf3yQP7tikx+6b%Cw$J^)AC)F=^*4$?7S3MHf<}(2i9W z`2cBBJ2bzb=a(97xW0IJKUv``JoY*%Y|KC!YzHEkmnidR)9*+xMFiCB6kDGJub4j> zYKFl@f4GM|^!vQcj77f~=Jgf}`O=NC0_q=%4tOI7UKgV&#VY8a1eeU=Fqwt~U52#P zGcYPmORS+D`A4Oe6A^a?;L`5TaVuh+K1~8&GZ@UdRLeyd^H2gjqcB070}4F^&Ah+J zyq*nEOPizJh9sxQ5_rrl6oY@JxB(Zv!d?Mw|JBG+VtLj7!V*GOo=_uvTzI3TIphOd zt&#&%@XRi5zgAU$xg^2)qDfj0vYFX45x#q}Q}xOnvmWzG%odPQuWa64 z*GcMgPqN2vLW8*Y$`h)t5;^M}h)KI2BXq>8>!k;P@)sW++IRbl!(!7bbk4;1$~1zpSc*I~9-uWli0T2+V2Y_p%VQSA@zC+MY6d($UXD zxoT!V3Gv6Hp3akH`%pe#c*3po2mNBdP))3*CeAkfLkY94UXYb4TkTML2Ccct;6~5} z@z`ro6c~UR>s8|bO`{$Ip!a>Yjgzn2yZI=iT6#WXSJYYj3cwwbsZ=iCHQmxS%&@L% z$Cal}wN+G95)|^}AgDlr7^4S&t4pY>^I(v^36CYj44th48#gh~8Xsw(7r8Ed!Ii8_ zs>)@6uwHXgpsO7-j0OA}ARR#LN$UI^xJX*XvC0y;ctJJRyj1izxS0!a&?9+Q>aQPZ zPf6ntoi8+Q@7 z$dZJ_aUyQKO)qcJ|7&7bql ziL=rE(crncj~hYB0A^-U8eD+SHxZG7gn@o8XyFi{Wc%-+rTz)Q&400#COB4PdHA{S z#eR?+MSl~q9M-)|4zDzM4Hd_5CxYqnnPEIf#F2r_@@|6LCmfDbBSNtz1Z)$1YNN2i zy6Wo7-^pr@eI?(9pPB50#M#?37Z&_MK4QF<+g9*_7Ep zhzx6h_L6x$onF0aW?On47o%|A^H|`T|IqS-_0=zNyDb4x4+71wkvvhL04xbJ=x6#4 z)e`Hj7j}$z-dAzCj-~4A`g+wy#?LEI6bG+51#ly-z>RcK-}I!n%R&yprNFm}iE`2L z6!{naQdxk@^wk6>Wi{VSpNV<66cTPF`{vzBK6*tGwG;>C7+KcRd73J6Uziy|y|5K^ zNQy_IaU%)iZyFQtHbdoBKtoT1g&(+%H)6r5t0%Z>J02ZnK`(uxGVm%^zS`mi>!U`cEt%6GLoVmPc*@NB&Bdv0ZJdn{$Rfl5fRWC?(9CU4>Z?FOY-z zln(&A-=YHPc6B}b*q=oK18}!=;lC$_?R{FqBfhROA95kcw>6x65`ifvXsw8SX~avP zTK#jRj(Hq3R3()WVL^y$|B|>O4=MKDNwp-?{DKsJm$1k~CV zv+F{39}~uh;5~nSH=C4duABOm%U0yR;z9Lh&R!29JU=J;f0==UJH+7m6i^DR=f>H< zWmgGDLO03ae4lsJ*BC^`p818W8wt(2XY$hk*KVK&|xzr7Hh2$DOReE3)@>l0S2ykc59}e7Gx&e=+G_8V?iZ zr3^7YXZ|mhhd^HNow|{A6tG?ft^@SNV?VvuLosj$Z1&M-2so}1jG`v-snJ7ZAl`3m zr4$1EVbKj`C!u%8(r<4fBmWF3CH-e63P%ad*L^ma!$$%;q_rYOZU6Z}IrX>X)5T+0 z4ker&kbP zC57uoc;~Z=UN;eV<2uAR`AbrkrQWJ)eNuAZ6dm2LwUuv0zPSziV$qshK&TM|YIQqL z7B30o+>JtCVWxvo(bBFT2aRzebm!OvrERg52y)AdZ_%d+CG6WH%nT_DnxE#w`1ev& z66_e(N#O~WCJZb+j^C=J)ku1TN9;?l69Vuh{N*4y#Q3f5e6jr9dd)Rxf5@GQ;C2GG z@><9y7NvTRbfPqTC#4WaJ3G6$w$&=An<&_J-WcXZZ-?jbdc6uKYgB0B&#f2v=f`&^ z5@!NuTG;GSuvCcBypq|qALPAxS^XmixVE!01FG&M6t0yU22nbEGZiW1WZd*LKMazU zi+M$dB?cy8{gpzrB5qUa%lsdSf&^vLfrr46Re;meLQ-FLsN_M1Od;|$ z9d=9m&6%m?I!KLJt3d(VWOs9#DUdC2I;f_!c;6#}w8cnUJSESh6!H6Wp}-dp@=5l^ zad6N`+R}KT#~L|q@>)Ko`iBVus^sKW76FBbt}x(hsh_`ivdK1rC&Jn_KZ^kpy8_dBfU_KJke99RP(b3P-Itb{cV?4Wxvx+fg z^V1TX7>w8POk1+;&Iy-(NN7kok?(Y`~83ydH_e9A z`4L3(p$}nvkDJ}8{#6e={Z3=0Y6LkIM>>CYMO-b}N9**aiFx`g%I+G(r}Gj1$%EG7WEx=-ZPc8^Au#IzYVl^kPhenVVEDQ+&Cat;Hg1pplfWfQjc)oy zX+)uri*C`=so#uQQvTD0^}@o}23SB)F7^cRvAF*qYThR+<&1=Q_FfD<0M85tL%O}p z*-GM_ki;4+p0B?sm^q~fDt&xbLQa%U`l;LMvgE!{SN2KB?EX?O96S(-0^s_Nm(b^U z?CWmkZ=iL@+|Ec9{An4@hYueDmQ*e7gY9bo{)Lk-i7C5mhSo%j)8wEj%i+MS{=3_j zVcx7nYzHj&sQW?~Ao5-zkorpg(cJTn1E6G`Q2bhCGf!H)Da}RK+@s!ptxPc-ILv1V>wWD2;TZ)(z^t4SE}8{U;-CES2`L z?fX!_qlrU2Z?Asu?rYxOFNy*z!#fXk87m<(47njI%O8XqVKN-beNhE1U$bBRw@fsP zhZdV%;;;(}@+p2i@Fx7Kc}{$&@ptse)!r6ME5#DeSEJfUD2ilekM&ZlMG1n@LVz~*v^;)k4c$uT=$tYu_JOZWi%?Yzq>ABV_iy2RzER3 zuM=i^eV*nV#mYfx99NxUwv=82B)RWwwGBxD49gT)p-C^|&g64*Ihe#^@F6-OoCBm! zadnwp22S7Epw@3|v8{O6G7AzciA}|He!rr}IV}V}#TOvdn8&SXCjh)g0UpY>mXRkD zU(D4Y=|4%k#HaRtgi-0Jv-3Q6;KXJJU`6WbUpV(fh*3|HCh#>!zTD5qDUMIQ zO=V<8xRIgG%s2qID4hwZk(@%MF1ci$#ynATcbvp`X~~E{VGqgXOMn3+J5?Q)L(R|Y zozqNC0ck%We$7%Ui04cWJ4}Z5ec|ZsxpJY1Xth1S+dRn;6AwAaJ8qGx6b*=c$~Rds zbJd^I?KGP&c!3GHL4+FA;MNd^W&o!sy}5S-FJgY4zEf9l{0Q+4(rVwq4T;_KjTZD> z%6PwF2lhF;?`Byqgr0q--^d}=5Y60!U1aeVt-jp&VA{o1`7Uj2N8Ojd2D+JsuRcj~ zKlcsPPRq(>YpoK6@7l&2uU)(k zEGP_)KeqoeKPQf<-XcO#Wl}}sY+lobkr+af@gn4T)C!3A?h_>=sTEGMD z`TXD4xH`OJwjb!RVzXa?TkDLh9kOAZf<6|I65`3g3=v+ZXp5ZgCsKWY`Dc!`=<`*b zX6-#^HkPhz=MAOfMTlK$MXNb>KKADTfPSQ1Dl{vsxxd2WuKmju6Qv`lR9t=V(eB03 zx12`0QWax&WWH4WlyipnO#W)V*h&|)or;~3vtPzBiSAxhZeaXEQKZiF$WV?j7XLSE z*!;YR5ME^l#WlO_d}5R#*=^{6IfX6uW{Hc(y?A>AzI9GPCxx>#(xTJNmB|2NZLM4V zkb6XY_FXnczL=QYmdS^Xn%j>vTW!h>2-wudcfeKGn8bDnsONWNS4!O^g2VT#@B;d~ zNqG^T+fVXVyIcKpY#ZS^)}5gjk_-_W*9%*XlLecU(H*>>+2+T{;!QH34JvmiGT}kr z`z9%G1&BnPOB|j4&tL`(#p;qyyOq}ORilAwkkCz%f`V?1-StPRu+IHywvzGrkrY3+ z-}&_S0Dd@fL9c@}*&Fbm#% zuV#O8bNx6jDlY3`gD zcKI68g#{-!sC@9f+})_%;u~x8`(FJOSIfGXJB>YzWQ!gHh-yaZk^4)nZl;><;y@+- zJyKR(mOyzm;KD8|8~iWf^iNnTlQ&wFd1Du|=z(So9Yd?a$?)~&9{5ZJ?5JmrnTscx zP(!x*i<9L}S5MCezqvB2jZEXrEb?`+d!UT3s$ny$vno*2nGBE=KygrL@XS4_EzT`a z=$_%EEVvo5Wiuo*&~>&_6ecrJz^-Vjs*-HXPrwz!D_MKaETXAOWOx)2?ay-g#ZRHU zob`~1j7*p6zobwH)YJNo2db_KcJGSw)ePV06&mlVq>kZ|UQBoEjfDi3su6Hk?)LP! zgpa0~ZZ+NGVytsVgwF8R3H82+dynw zbMgD@nUYlqHC4Sz=7LAM;l(a*B%kqVKrXt@JJZUXG_Svdfja{d2RIH^dR&x0zjZ9X zuYQziL>hzZVA&^ksgGL=^aq{X7vk7I0}64LdF^S3QO~abQ}=BKY7HB{&CTtV=qQ(k zTsNqV?*)&MvJrcz^L6(0Yvzru94WSc z@R(W|BnGNht#Ojh*UOj7tQw7btWKL$WjjHNLv*k10u><=0CLg3Y#uPZBII|}pi8CA zmRZ}E)gxS1Cihk; zsrBoSicb>O*!&zms)+61n81xLvLP(5e79RT{IS#5Y+U9XyOV1Yxy6Ljytda89^h!> zs#tzYb3B{JCfiWl#}hW1VCLjz5%W)SBWtdx18}Q7-z1o10Ek6UQE?TXa5rfs<=)Sz z77AOgvJfrZTDvchb)(Su)$rNz9IVgVtYYKkHzl6An2|xQtDCuBBhm}~&ZVPMeferZ zcq-=(|um6c^|77)m2!- z-_xDM&C=`85M=0SeX4rF-6l;dJ%@>7d=Z~TAL%Kenk6|X8`D*hZh)ZQw1xQpN}sb0 zuv3CiF*+kHGQJkGc`4Tivyv z_K_W#<@X6*>TYb@HFc0=MOZLBq}Qa@#|XD9eHzI_`yENKbdXz8q~b{;UonepyM`HOG5k+%uOQH zd@myXX&;moy(RlmTp_b%lX5D&?`z`pJMW!t9ppBA?``jC68&7COJ)u%eq|XSiS`Mo zNbe3fU|!m~jP@^lZbO8+fFMP?EqQmrR0FYDpCSSj*+&hKBSEG#({NqhC37XLBa2ti zl1D5+Y6!qO;UGSP4%h4RF7k#Byqi+O|ONlrjy+_;gA81Cf z5!$D3JG9kEy-;P5foI$|69&ok*^2GulE!w z{0W0|GF%?QhzF1|tNF@qN(=%BO z8HksMv<5-QzSz*?x2mY})cY2P=cfRm#h$@L z`aLGxnHF#eJN$=;cT(WM(BBJ7Ho3)DzJ#KwFaf&y|B?L!x|09Xo&0}Ki~jq-nMgw< zb)pDpNci7wk6=WUV}UTHF+r^-FQGC^K>s5BeTtV5HGSXK@c+_Vk?M`KyQHtnXrAq( z0NO}ocEK}6G-Vc$??CqeW+za+0I0|R%`WRjRzzxK6Kph&7%Gcvt<5cIukpb_MF{smv){jAr_J$#5&*&Qb; zy8NB&P-mCm7e;GYvbY2U4_}PF|DkjD`O8V10UZ@>;qIf+n)~h-JL2DX4zJkZSd-c8 z@iC4ZP8N7dmjHkh_}mTVq_$x=fLtX7>gfTn(?qh63>D|l(jSqf#4+q7gX0h#!pdlqs@>8%?h)1|0Rt`L{LY4QBphomx^E(B=b)5|>S z!f84|8-UG61O)K=Q}}~VPo_c3uDo_YUpf1&=-!k=5ABMe&+|AaX}61VDb+Py@7X8U zCcvqPU{r}hl7jD%VhlYQUf==L_l2k-iRgdMQ;ZE^aDRu;!~607v*~_>7&(|%;i0zd zO_vf0MtN_cBQ3ha9|Gswk_X3gFwj(b`g@1|m|%#>Z|e7pZzAL^moLX*7pucGPd!$y zIS<4l3L0YRPQIHz^>}AT2gs!C5Li$t-r_663?U0?J{>hf1+-@`q?ZFx6c>Jb$UaqX zokoaibek@^*y_h48cEtaJeP;Oe<;8M0JRTmTs=9IMX~vLRxNbh-;=^Kags#s1nM@s z_KGT2t9Vov*E7gb272Rf-*F@I#)$i`mHOFv5!O2$3=xh`Z}eI^ev?58pcA^0?V7Epm#GgLUisZ%`)EMBKjLn0!aAb=^s8Uc&DF0Wu_l}N z>leoh=FhB+VHX?o6Abu@uUGz_#*FMNhRzXnf@g)mgD!^Zp!{nWhntKLe=n;G=DPkj8shtb-cG1ta-$4+Aj(&-Ip81 ziTOZB!iD%A&jP=@Z2-$J_==xgeR_Iv`soA@Z|F(u^@qlAUk7d4q9&>EANAkEDQql# z+-?}tE^kE`F0HEr^2&WQOPIEFNlWfkqkIe?cj-654eCab;*C*UlvFvX@ue|!x7IoR zT8pu4EIyG~bbIU*$vbQgr2o0~1%I!=NgJ|3m zDyBoDB9}}FF^58G3h6R4CSpoCs#OY{`mg; z*7w)D)?R!6_Pf`5-@TvT^V`ou`^4-EPYhUjEqXKDT&?^PR=+~4XW6xM#}+uveq0R& zrqFNI*p_+{@w{~Ryh?_6-78oP-0r)ou-Ru~@Ob<%RpKve5`esbAbw*KlUs@!((@=! zs~gDODB8M@URZZ(tzBxeyvaM#RV4Ml@pOaiJ*}KwkmN(n}$VQXyBb z&-(Y)ikFMD9i;Hw*znfx4X!|pc8^6zXd2J~l3kxhvchkrc*-+t4|{FV;DREvxdK0i zcR{hix$Nxcnx6W|s-Ft{#;x0IY-z?XuuNC$vpN**pk5g&nEnD>Rbmxo50#1?_bNE4 zDP0Y{j!j=*qVy#Q89&PYy zaapJ$%EBLrdpWl@YObaz*e2A<3@HQasSUVmNcVuMoDbdIew2GhJ>@RD4#!iW$}5=L zvS#NQ-g5La=SELBV38iv`ebjV3mLI4rhJ2@(0KN!F*$J8pQciKfbVz z_m8D6Y!4bx`g!vEsv0^=-c%&>)C;&D0fGUvyR60@zB=h%^iihV)rdVdh6)_W2R#>M zFC6m>I~3=GQch6gBjc3vdH3Y2ZTAkxc?U0s;IKFnqhQe(LDuEQbJd!ngECn=mbwzF;%Be>h88Jqrtj~IV$RTDub zZU1<(!(|^~UGm-MDR%!KbuFi=Cq%c>C4MvB+?|PiTWH{C_8j?JN%MHk{10x;6dJ_W z=aFeUYu+B@ZTGsU0QN8t%>T4yy0#}EfmhzXcfw6Hx7pl3lS6OKAn+-3Ms9H>f=09(^tN!i$m^P%a0_D_PnEdc{QguR6329K6EV z{Vg^3B2EgP8Igx3Td>8T@N>A^g-Q^4#%Myyu?LgC8O~4ja#BSq&bj*sTvhUL*B$$e zV68^NEOEl>4(Aq)!+;+5R&Bq)AjJ&`do5%8@w7L9nF`vRcH!B+w!J#6@o{7q-;nm?uqH9D$$8xv9AK+OXs%jMI^{DrrMWKVlC71c{*cWOAi&b=kOb zIwNMHRXSNZJOJ1454QMlfH{nE&t4gI@`drF$*beGK?>iZE7SCNC{BI`UljxjWl*M8 zeE_gJmz#d&z%u3xT(M-9kT{pEL=cR#dYu9FVAL}lUU0haU42LB?SvxW0%bPz{ZkA9-+;~ zi2OIm^{G;@1<9Wlk6WbXS3ycxKdz7q8#PJvmyoWcRgcp-W!Gr7uw z6D4)ph7v!qtr8w0!~zb<_PH&>ePfNXw91%A9ek%mew^~Jk}?9%a~FOj@II}_ED-Q7 zLI{EzMQnI~XEqv-8Rz9iX18tN+02BKz>S>xw5Y2Nby7t&cA7mwb|)!{*R2zirE?!U zs^NZ~Mq{e7X;ExH6J=uXRzkJ{xBZWIFhP0ja>0jXnW)|B79_~}$Pl7ofcMPj?D{FW zb{FfdxHETcVsS(ptH)M<9G)YofpGJXz5Oh&WR)*{9ipVbm7rAipQOW$NRF71{5F!>1cDpcju9VoqQ}9yBRe(|JFELHI za7@Ad{a4{C+zrZYCu@yOKm4T<%7`y}LV;>1nmapPUVP9u=-P1BK;RV79>0}KR#N%l zGdOvNmW8st)JWVeYusQ%l$&O&ez~-~fk%~HIbSqqFj~w*&}{l@-Uo=lcc(X|Q#NrX zj#hYVC>@LMx(4N1uUJX7x2>^mW7*{}UytuA!7GlodJ@U&9oh|&Y%~t8SnvUqwvA@` zVO)`IowL5u!C4P!SM81>^3>dtCxT?2I`Oy>Q*6dUvjXDXK>W+l&d#?QA_QWrD}vwj zL5xdT20wqghlR~rg=^)F<}`#$(wWt1m?4Q}AfPhX%89x)mqf3u6wxRtq8aChlbpJS z8g&0$<;UT=-1+nEEFr%_Ay4XC-(rpMuv%GLa)z3o01A<= z|4dS&JXDtR^l7b#t>>l{y66@?dK^my(-e|+X2DJIX?+SNWL+T9c%K7vlojN|#@mFx zNj0JbKBijWD@TPpDoc-t$67W@OFQsiQ!&N0^7@&j5Q5WP?A5P=nQ8Q$$T1o^iCv|M~(fH=vj_QPh@p;bFiJVG$ZYu zYP!V~-TM_!Bg4kJ&)NxYnro9cyPK}^ROBG*W~U1t?;u;?4ENV{O>Ep5s`4|jx06oY zf+WHIyqbSx95Nra>7vcnv6;f-2dQW-wjhDVaMkI`tup9`7`Nvq{aBEXcA-y}5^u*{ zUVGCVFja@I5}iK$d3@qG#Mre!!a+iu?{CSG%2z6QxNW-d=FR4J0swb7yV+dZGviDd z!RsB;u7X=_E2|M`LE*uD1CDi9O}eTxA8!KxMS4$gl10%zM9#QR+6*t^bR=UsxZ(ay8eU9-=ALR$j#{G2wn5GXT8X;NUd($ zrS0a+6$YvM4V;s!AW)fdWJCKcv}1vHJdtLRC6A9%Eq%=BnVUAUz@)`8tUD7Og|~j5 z*4AU_bL$*KQQ2XO?KvpUR{`gXt1f&3zdCyr%u*p>PyMJLk0Kgreu+k5QD$)+It-!% zg+vJx?cWQ=`t@z0Y-T}js@V`FxU%N)>hUhKLu&8;Rqq&)sqi zqSCuAbrW&Ko?Q<+8vM4{>O5wU{X&$_-p_`VGlx<(>S8iT?`=?A{)+lwVp5M$-GJC{ z%K*ww@2|85vAfyQ=-^Ki24J)UTbGKla{A&1V+VtII8vjStoGp!eSr;6HErE7{i>V1 zQp+PT-zj=axIkqjn#3E(?Ucv?s@xL*61N}Vco&LhGe(<*X zvd8NQt^bq7-M^=)y(}RVB<{3HzLt6pY~6`>3&uyD!k>1CxNsVLY(fG^8w7wv0DA*% w>>OOTgYkBdBuOaAWzrk?B{+nCqW?Z~0(gV@Y}0)z*lZ2$lO literal 0 HcmV?d00001 diff --git a/rust/ballista/docs/user-guide/src/introduction.md b/rust/ballista/docs/user-guide/src/introduction.md new file mode 100644 index 0000000000000..59d7a1a2a5c12 --- /dev/null +++ b/rust/ballista/docs/user-guide/src/introduction.md @@ -0,0 +1,52 @@ + + +## Overview + +Ballista is a distributed compute platform primarily implemented in Rust, and powered by Apache Arrow. It is +built on an architecture that allows other programming languages to be supported as first-class citizens without paying +a penalty for serialization costs. + +The foundational technologies in Ballista are: + +- [Apache Arrow](https://arrow.apache.org/) memory model and compute kernels for efficient processing of data. +- [Apache Arrow Flight Protocol](https://arrow.apache.org/blog/2019/10/13/introducing-arrow-flight/) for efficient data transfer between processes. +- [Google Protocol Buffers](https://developers.google.com/protocol-buffers) for serializing query plans. +- [Docker](https://www.docker.com/) for packaging up executors along with user-defined code. + +## Architecture + +The following diagram highlights some of the integrations that will be possible with this unique architecture. Note that not all components shown here are available yet. + +![Ballista Architecture Diagram](img/ballista-architecture.png) + +## How does this compare to Apache Spark? + +Although Ballista is largely inspired by Apache Spark, there are some key differences. + +- The choice of Rust as the main execution language means that memory usage is deterministic and avoids the overhead of GC pauses. +- Ballista is designed from the ground up to use columnar data, enabling a number of efficiencies such as vectorized +processing (SIMD and GPU) and efficient compression. Although Spark does have some columnar support, it is still +largely row-based today. +- The combination of Rust and Arrow provides excellent memory efficiency and memory usage can be 5x - 10x lower than Apache Spark in some cases, which means that more processing can fit on a single node, reducing the overhead of distributed compute. +- The use of Apache Arrow as the memory model and network protocol means that data can be exchanged between executors in any programming language with minimal serialization overhead. + +## Status + +Ballista is at the proof-of-concept phase currently but is under active development by a growing community. \ No newline at end of file diff --git a/rust/ballista/docs/user-guide/src/kubernetes.md b/rust/ballista/docs/user-guide/src/kubernetes.md new file mode 100644 index 0000000000000..8cd8beeb267e6 --- /dev/null +++ b/rust/ballista/docs/user-guide/src/kubernetes.md @@ -0,0 +1,216 @@ + +# Deploying Ballista with Kubernetes + +Ballista can be deployed to any Kubernetes cluster using the following instructions. These instructions assume that +you are already comfortable with managing Kubernetes deployments. + +The k8s deployment consists of: + +- k8s stateful set for one or more scheduler processes +- k8s stateful set for one or more executor processes +- k8s service to route traffic to the schedulers +- k8s persistent volume and persistent volume claims to make local data accessible to Ballista + +## Limitations + +Ballista is at an early stage of development and therefore has some significant limitations: + +- There is no support for shared object stores such as S3. All data must exist locally on each node in the + cluster, including where any client process runs (until + [#473](https://github.com/ballista-compute/ballista/issues/473) is resolved). +- Only a single scheduler instance is currently supported unless the scheduler is configured to use `etcd` as a + backing store. + +## Create Persistent Volume and Persistent Volume Claim + +Copy the following yaml to a `pv.yaml` file and apply to the cluster to create a persistent volume and a persistent +volume claim so that the specified host directory is available to the containers. This is where any data should be +located so that Ballista can execute queries against it. + +```yaml +apiVersion: v1 +kind: PersistentVolume +metadata: + name: data-pv + labels: + type: local +spec: + storageClassName: manual + capacity: + storage: 10Gi + accessModes: + - ReadWriteOnce + hostPath: + path: "/mnt" +--- +apiVersion: v1 +kind: PersistentVolumeClaim +metadata: + name: data-pv-claim +spec: + storageClassName: manual + accessModes: + - ReadWriteOnce + resources: + requests: + storage: 3Gi +``` + +To apply this yaml: + +```bash +kubectl apply -f pv.yaml +``` + +You should see the following output: + +```bash +persistentvolume/data-pv created +persistentvolumeclaim/data-pv-claim created +``` + +## Deploying Ballista Scheduler and Executors + +Copy the following yaml to a `cluster.yaml` file. + +```yaml +apiVersion: v1 +kind: Service +metadata: + name: ballista-scheduler + labels: + app: ballista-scheduler +spec: + ports: + - port: 50050 + name: scheduler + clusterIP: None + selector: + app: ballista-scheduler +--- +apiVersion: apps/v1 +kind: StatefulSet +metadata: + name: ballista-scheduler +spec: + serviceName: "ballista-scheduler" + replicas: 1 + selector: + matchLabels: + app: ballista-scheduler + template: + metadata: + labels: + app: ballista-scheduler + ballista-cluster: ballista + spec: + containers: + - name: ballista-scheduler + image: ballistacompute/ballista-rust:0.4.2-SNAPSHOT + command: ["/scheduler"] + args: ["--port=50050"] + ports: + - containerPort: 50050 + name: flight + volumeMounts: + - mountPath: /mnt + name: data + volumes: + - name: data + persistentVolumeClaim: + claimName: data-pv-claim +--- +apiVersion: apps/v1 +kind: StatefulSet +metadata: + name: ballista-executor +spec: + serviceName: "ballista-scheduler" + replicas: 2 + selector: + matchLabels: + app: ballista-executor + template: + metadata: + labels: + app: ballista-executor + ballista-cluster: ballista + spec: + containers: + - name: ballista-executor + image: ballistacompute/ballista-rust:0.4.2-SNAPSHOT + command: ["/executor"] + args: ["--port=50051", "--scheduler-host=ballista-scheduler", "--scheduler-port=50050", "--external-host=$(MY_POD_IP)"] + env: + - name: MY_POD_IP + valueFrom: + fieldRef: + fieldPath: status.podIP + ports: + - containerPort: 50051 + name: flight + volumeMounts: + - mountPath: /mnt + name: data + volumes: + - name: data + persistentVolumeClaim: + claimName: data-pv-claim +``` + +```bash +$ kubectl apply -f cluster.yaml +``` + +This should show the following output: + +``` +service/ballista-scheduler created +statefulset.apps/ballista-scheduler created +statefulset.apps/ballista-executor created +``` + +You can also check status by running `kubectl get pods`: + +```bash +$ kubectl get pods +NAME READY STATUS RESTARTS AGE +busybox 1/1 Running 0 16m +ballista-scheduler-0 1/1 Running 0 42s +ballista-executor-0 1/1 Running 2 42s +ballista-executor-1 1/1 Running 0 26s +``` + +You can view the scheduler logs with `kubectl logs ballista-scheduler-0`: + +``` +$ kubectl logs ballista-scheduler-0 +[2021-02-19T00:24:01Z INFO scheduler] Ballista v0.4.2-SNAPSHOT Scheduler listening on 0.0.0.0:50050 +[2021-02-19T00:24:16Z INFO ballista::scheduler] Received register_executor request for ExecutorMetadata { id: "b5e81711-1c5c-46ec-8522-d8b359793188", host: "10.1.23.149", port: 50051 } +[2021-02-19T00:24:17Z INFO ballista::scheduler] Received register_executor request for ExecutorMetadata { id: "816e4502-a876-4ed8-b33f-86d243dcf63f", host: "10.1.23.150", port: 50051 } +``` + +## Deleting the Ballista cluster + +Run the following kubectl command to delete the cluster. + +```bash +kubectl delete -f cluster.yaml +``` \ No newline at end of file diff --git a/rust/ballista/docs/user-guide/src/standalone.md b/rust/ballista/docs/user-guide/src/standalone.md new file mode 100644 index 0000000000000..e4c24fedd3198 --- /dev/null +++ b/rust/ballista/docs/user-guide/src/standalone.md @@ -0,0 +1,92 @@ + +## Deploying a standalone Ballista cluster + +### Start a Scheduler + +Start a scheduler using the following syntax: + +```bash +docker run --network=host \ + -d ballistacompute/ballista-rust:0.4.2-SNAPSHOT \ + /scheduler --port 50050 +``` + +Run `docker ps` to check that the process is running: + +``` +$ docker ps +CONTAINER ID IMAGE COMMAND CREATED STATUS PORTS NAMES +59452ce72138 ballistacompute/ballista-rust:0.4.2-SNAPSHOT "/scheduler --port 5…" 6 seconds ago Up 5 seconds affectionate_hofstadter +``` + +Run `docker logs CONTAINER_ID` to check the output from the process: + +``` +$ docker logs 59452ce72138 +[2021-02-14T18:32:20Z INFO scheduler] Ballista v0.4.2-SNAPSHOT Scheduler listening on 0.0.0.0:50050 +``` + +### Start executors + +Start one or more executor processes. Each executor process will need to listen on a different port. + +```bash +docker run --network=host \ + -d ballistacompute/ballista-rust:0.4.2-SNAPSHOT \ + /executor --external-host localhost --port 50051 +``` + +Use `docker ps` to check that both the scheduer and executor(s) are now running: + +``` +$ docker ps +CONTAINER ID IMAGE COMMAND CREATED STATUS PORTS NAMES +0746ce262a19 ballistacompute/ballista-rust:0.4.2-SNAPSHOT "/executor --externa…" 2 seconds ago Up 1 second naughty_mclean +59452ce72138 ballistacompute/ballista-rust:0.4.2-SNAPSHOT "/scheduler --port 5…" 4 minutes ago Up 4 minutes affectionate_hofstadter +``` + +Use `docker logs CONTAINER_ID` to check the output from the executor(s): + +``` +$ docker logs 0746ce262a19 +[2021-02-14T18:36:25Z INFO executor] Running with config: ExecutorConfig { host: "localhost", port: 50051, work_dir: "/tmp/.tmpVRFSvn", concurrent_tasks: 4 } +[2021-02-14T18:36:25Z INFO executor] Ballista v0.4.2-SNAPSHOT Rust Executor listening on 0.0.0.0:50051 +[2021-02-14T18:36:25Z INFO executor] Starting registration with scheduler +``` + +The external host and port will be registered with the scheduler. The executors will discover other executors by +requesting a list of executors from the scheduler. + +### Using etcd as backing store + +_NOTE: This functionality is currently experimental_ + +Ballista can optionally use [etcd](https://etcd.io/) as a backing store for the scheduler. + +```bash +docker run --network=host \ + -d ballistacompute/ballista-rust:0.4.2-SNAPSHOT \ + /scheduler --port 50050 \ + --config-backend etcd \ + --etcd-urls etcd:2379 +``` + +Please refer to the [etcd](https://etcd.io/) web site for installation instructions. Etcd version 3.4.9 or later is +recommended. diff --git a/rust/ballista/rust/.dockerignore b/rust/ballista/rust/.dockerignore new file mode 100644 index 0000000000000..96f99a522ad0b --- /dev/null +++ b/rust/ballista/rust/.dockerignore @@ -0,0 +1,23 @@ +# 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. + +# Turn .dockerignore to .dockerallow by excluding everything and explicitly +# allowing specific files and directories. This enables us to quickly add +# dependency files to the docker content without scanning the whole directory. +# This setup requires to all of our docker containers have arrow's source +# as a mounted directory. +target \ No newline at end of file diff --git a/rust/ballista/rust/.gitignore b/rust/ballista/rust/.gitignore new file mode 100644 index 0000000000000..97eec16404695 --- /dev/null +++ b/rust/ballista/rust/.gitignore @@ -0,0 +1,2 @@ +target +temp \ No newline at end of file diff --git a/rust/ballista/rust/Cargo.toml b/rust/ballista/rust/Cargo.toml new file mode 100644 index 0000000000000..d1f588f3bd75c --- /dev/null +++ b/rust/ballista/rust/Cargo.toml @@ -0,0 +1,30 @@ +# 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. + +[workspace] + +members = [ + "benchmarks/tpch", + "client", + "core", + "executor", + "scheduler", +] + +[profile.release] +lto = true +codegen-units = 1 diff --git a/rust/ballista/rust/benchmarks/tpch/.dockerignore b/rust/ballista/rust/benchmarks/tpch/.dockerignore new file mode 100644 index 0000000000000..3a7d0fdaa06f2 --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/.dockerignore @@ -0,0 +1,25 @@ +# 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. + +# Turn .dockerignore to .dockerallow by excluding everything and explicitly +# allowing specific files and directories. This enables us to quickly add +# dependency files to the docker content without scanning the whole directory. +# This setup requires to all of our docker containers have arrow's source +# as a mounted directory. + +data +target \ No newline at end of file diff --git a/rust/ballista/rust/benchmarks/tpch/.gitignore b/rust/ballista/rust/benchmarks/tpch/.gitignore new file mode 100644 index 0000000000000..6320cd248dd8a --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/.gitignore @@ -0,0 +1 @@ +data \ No newline at end of file diff --git a/rust/ballista/rust/benchmarks/tpch/Cargo.toml b/rust/ballista/rust/benchmarks/tpch/Cargo.toml new file mode 100644 index 0000000000000..55a0fe1330cfd --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/Cargo.toml @@ -0,0 +1,37 @@ +# 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. + +[package] +name = "tpch" +version = "0.4.2-SNAPSHOT" +homepage = "https://github.com/apache/arrow" +repository = "https://github.com/apache/arrow" +authors = ["Apache Arrow "] +license = "Apache-2.0" +edition = "2018" + +[dependencies] +ballista = { path="../../client" } + +arrow = { git = "https://github.com/apache/arrow", rev="46161d2" } +datafusion = { git = "https://github.com/apache/arrow", rev="46161d2" } +parquet = { git = "https://github.com/apache/arrow", rev="46161d2" } + + +env_logger = "0.8" +tokio = { version = "1.0", features = ["macros", "rt", "rt-multi-thread"] } +structopt = "0.3" diff --git a/rust/ballista/rust/benchmarks/tpch/README.md b/rust/ballista/rust/benchmarks/tpch/README.md new file mode 100644 index 0000000000000..a07d29f854fae --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/README.md @@ -0,0 +1,103 @@ + + +# TPC-H Benchmarks + +TPC-H is an industry standard benchmark for testing databases and query engines. A command-line tool is available that +can generate the raw test data at any given scale factor (scale factor refers to the amount of data to be generated). + +## Generating Test Data + +TPC-H data can be generated using the `tpch-gen.sh` script, which creates a Docker image containing the TPC-DS data +generator. + +```bash +./tpch-gen.sh +``` + +Data will be generated into the `data` subdirectory and will not be checked in because this directory has been added +to the `.gitignore` file. + +## Running the Benchmarks + +To run the benchmarks it is necessary to have at least one Ballista scheduler and one Ballista executor running. + +To run the scheduler from source: + +```bash +cd $BALLISTA_HOME/rust/ballista +RUST_LOG=info cargo run --release --bin scheduler +``` + +By default the scheduler will bind to `0.0.0.0` and listen on port 50050. + +To run the executor from source: + +```bash +cd $BALLISTA_HOME/rust/ballista +RUST_LOG=info cargo run --release --bin executor +``` + +By default the executor will bind to `0.0.0.0` and listen on port 50051. + +You can add SIMD/snmalloc/LTO flags to improve speed (with longer build times): + +``` +RUST_LOG=info RUSTFLAGS='-C target-cpu=native -C lto -C codegen-units=1 -C embed-bitcode' cargo run --release --bin executor --features "simd snmalloc" --target x86_64-unknown-linux-gnu +``` + +To run the benchmarks: + +```bash +cargo run benchmark --host localhost --port 50050 --query 1 --path $(pwd)/data --format tbl +``` + +## Running the Benchmarks on docker-compose + +To start a Rust scheduler and executor using Docker Compose: + +```bash +cd $BALLISTA_HOME +./dev/build-rust.sh +cd $BALLISTA_HOME/rust/benchmarks/tpch +docker-compose up +``` + +Then you can run the benchmark with: + +```bash +docker-compose run ballista-client cargo run benchmark --host ballista-scheduler --port 50050 --query 1 --path /data --format tbl +``` + +## Expected output + +The result of query 1 should produce the following output: + +``` ++--------------+--------------+----------+--------------------+--------------------+--------------------+--------------------+--------------------+----------------------+-------------+ +| l_returnflag | l_linestatus | sum_qty | sum_base_price | sum_disc_price | sum_charge | avg_qty | avg_price | avg_disc | count_order | ++--------------+--------------+----------+--------------------+--------------------+--------------------+--------------------+--------------------+----------------------+-------------+ +| A | F | 37734107 | 56586554400.73001 | 53758257134.870026 | 55909065222.82768 | 25.522005853257337 | 38273.12973462168 | 0.049985295838396455 | 1478493 | +| N | F | 991417 | 1487504710.3799996 | 1413082168.0541 | 1469649223.1943746 | 25.516471920522985 | 38284.467760848296 | 0.05009342667421622 | 38854 | +| N | O | 74476023 | 111701708529.50996 | 106118209986.10472 | 110367023144.56622 | 25.502229680934594 | 38249.1238377803 | 0.049996589476752576 | 2920373 | +| R | F | 37719753 | 56568041380.90001 | 53741292684.60399 | 55889619119.83194 | 25.50579361269077 | 38250.854626099666 | 0.05000940583012587 | 1478870 | ++--------------+--------------+----------+--------------------+--------------------+--------------------+--------------------+--------------------+----------------------+-------------+ +Query 1 iteration 0 took 1956.1 ms +Query 1 avg time: 1956.11 ms +``` diff --git a/rust/ballista/rust/benchmarks/tpch/docker-compose.yaml b/rust/ballista/rust/benchmarks/tpch/docker-compose.yaml new file mode 100644 index 0000000000000..f872ce16e2d8b --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/docker-compose.yaml @@ -0,0 +1,62 @@ +# 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. +version: '2.0' +services: + etcd: + image: quay.io/coreos/etcd:v3.4.9 + command: "etcd -advertise-client-urls http://etcd:2379 -listen-client-urls http://0.0.0.0:2379" + ballista-scheduler: + image: ballistacompute/ballista-rust:0.4.2-SNAPSHOT + command: "/scheduler --config-backend etcd --etcd-urls etcd:2379 --bind-host 0.0.0.0 --port 50050" + environment: + - RUST_LOG=ballista=debug + volumes: + - ./data:/data + depends_on: + - etcd + ballista-executor-1: + image: ballistacompute/ballista-rust:0.4.2-SNAPSHOT + command: "/executor --bind-host 0.0.0.0 --port 50051 --external-host ballista-executor-1 --scheduler-host ballista-scheduler" + environment: + - RUST_LOG=info + volumes: + - ./data:/data + depends_on: + - ballista-scheduler + ballista-executor-2: + image: ballistacompute/ballista-rust:0.4.2-SNAPSHOT + command: "/executor --bind-host 0.0.0.0 --port 50052 --external-host ballista-executor-2 --scheduler-host ballista-scheduler" + environment: + - RUST_LOG=info + volumes: + - ./data:/data + depends_on: + - ballista-scheduler + ballista-client: + image: ballistacompute/ballista-rust:0.4.2-SNAPSHOT + command: "/bin/sh" # do nothing + working_dir: /ballista/benchmarks/tpch + environment: + - RUST_LOG=info + volumes: + - ./data:/data + - ../..:/ballista + depends_on: + - ballista-scheduler + - ballista-executor-1 + - ballista-executor-2 + diff --git a/rust/ballista/rust/benchmarks/tpch/entrypoint.sh b/rust/ballista/rust/benchmarks/tpch/entrypoint.sh new file mode 100755 index 0000000000000..71c04324afd14 --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/entrypoint.sh @@ -0,0 +1,22 @@ +#!/bin/bash +# 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. + +set -e +cd /tpch-dbgen +./dbgen -vf -s 1 +mv *.tbl /data \ No newline at end of file diff --git a/rust/ballista/rust/benchmarks/tpch/queries/q1.sql b/rust/ballista/rust/benchmarks/tpch/queries/q1.sql new file mode 100644 index 0000000000000..a0fcf159e2094 --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/queries/q1.sql @@ -0,0 +1,21 @@ +select + l_returnflag, + l_linestatus, + sum(l_quantity) as sum_qty, + sum(l_extendedprice) as sum_base_price, + sum(l_extendedprice * (1 - l_discount)) as sum_disc_price, + sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) as sum_charge, + avg(l_quantity) as avg_qty, + avg(l_extendedprice) as avg_price, + avg(l_discount) as avg_disc, + count(*) as count_order +from + lineitem +where + l_shipdate <= date '1998-09-02' +group by + l_returnflag, + l_linestatus +order by + l_returnflag, + l_linestatus; \ No newline at end of file diff --git a/rust/ballista/rust/benchmarks/tpch/queries/q10.sql b/rust/ballista/rust/benchmarks/tpch/queries/q10.sql new file mode 100644 index 0000000000000..cf45e43485fb5 --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/queries/q10.sql @@ -0,0 +1,31 @@ +select + c_custkey, + c_name, + sum(l_extendedprice * (1 - l_discount)) as revenue, + c_acctbal, + n_name, + c_address, + c_phone, + c_comment +from + customer, + orders, + lineitem, + nation +where + c_custkey = o_custkey + and l_orderkey = o_orderkey + and o_orderdate >= date '1993-10-01' + and o_orderdate < date '1994-01-01' + and l_returnflag = 'R' + and c_nationkey = n_nationkey +group by + c_custkey, + c_name, + c_acctbal, + c_phone, + n_name, + c_address, + c_comment +order by + revenue desc; \ No newline at end of file diff --git a/rust/ballista/rust/benchmarks/tpch/queries/q11.sql b/rust/ballista/rust/benchmarks/tpch/queries/q11.sql new file mode 100644 index 0000000000000..c23ed1c71bfb3 --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/queries/q11.sql @@ -0,0 +1,27 @@ +select + ps_partkey, + sum(ps_supplycost * ps_availqty) as value +from + partsupp, + supplier, + nation +where + ps_suppkey = s_suppkey + and s_nationkey = n_nationkey + and n_name = 'GERMANY' +group by + ps_partkey having + sum(ps_supplycost * ps_availqty) > ( + select + sum(ps_supplycost * ps_availqty) * 0.0001 + from + partsupp, + supplier, + nation + where + ps_suppkey = s_suppkey + and s_nationkey = n_nationkey + and n_name = 'GERMANY' + ) +order by + value desc; \ No newline at end of file diff --git a/rust/ballista/rust/benchmarks/tpch/queries/q12.sql b/rust/ballista/rust/benchmarks/tpch/queries/q12.sql new file mode 100644 index 0000000000000..f8e6d960c8420 --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/queries/q12.sql @@ -0,0 +1,30 @@ +select + l_shipmode, + sum(case + when o_orderpriority = '1-URGENT' + or o_orderpriority = '2-HIGH' + then 1 + else 0 + end) as high_line_count, + sum(case + when o_orderpriority <> '1-URGENT' + and o_orderpriority <> '2-HIGH' + then 1 + else 0 + end) as low_line_count +from + lineitem + join + orders + on + l_orderkey = o_orderkey +where + l_shipmode in ('MAIL', 'SHIP') + and l_commitdate < l_receiptdate + and l_shipdate < l_commitdate + and l_receiptdate >= date '1994-01-01' + and l_receiptdate < date '1995-01-01' +group by + l_shipmode +order by + l_shipmode; \ No newline at end of file diff --git a/rust/ballista/rust/benchmarks/tpch/queries/q13.sql b/rust/ballista/rust/benchmarks/tpch/queries/q13.sql new file mode 100644 index 0000000000000..4bfe8c35553cb --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/queries/q13.sql @@ -0,0 +1,20 @@ +select + c_count, + count(*) as custdist +from + ( + select + c_custkey, + count(o_orderkey) + from + customer left outer join orders on + c_custkey = o_custkey + and o_comment not like '%special%requests%' + group by + c_custkey + ) as c_orders (c_custkey, c_count) +group by + c_count +order by + custdist desc, + c_count desc; \ No newline at end of file diff --git a/rust/ballista/rust/benchmarks/tpch/queries/q14.sql b/rust/ballista/rust/benchmarks/tpch/queries/q14.sql new file mode 100644 index 0000000000000..d8ef6afaca9bb --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/queries/q14.sql @@ -0,0 +1,13 @@ +select + 100.00 * sum(case + when p_type like 'PROMO%' + then l_extendedprice * (1 - l_discount) + else 0 + end) / sum(l_extendedprice * (1 - l_discount)) as promo_revenue +from + lineitem, + part +where + l_partkey = p_partkey + and l_shipdate >= date '1995-09-01' + and l_shipdate < date '1995-10-01'; \ No newline at end of file diff --git a/rust/ballista/rust/benchmarks/tpch/queries/q16.sql b/rust/ballista/rust/benchmarks/tpch/queries/q16.sql new file mode 100644 index 0000000000000..36b7c07c164a2 --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/queries/q16.sql @@ -0,0 +1,30 @@ +select + p_brand, + p_type, + p_size, + count(distinct ps_suppkey) as supplier_cnt +from + partsupp, + part +where + p_partkey = ps_partkey + and p_brand <> 'Brand#45' + and p_type not like 'MEDIUM POLISHED%' + and p_size in (49, 14, 23, 45, 19, 3, 36, 9) + and ps_suppkey not in ( + select + s_suppkey + from + supplier + where + s_comment like '%Customer%Complaints%' +) +group by + p_brand, + p_type, + p_size +order by + supplier_cnt desc, + p_brand, + p_type, + p_size; \ No newline at end of file diff --git a/rust/ballista/rust/benchmarks/tpch/queries/q17.sql b/rust/ballista/rust/benchmarks/tpch/queries/q17.sql new file mode 100644 index 0000000000000..1e65550634fa2 --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/queries/q17.sql @@ -0,0 +1,17 @@ +select + sum(l_extendedprice) / 7.0 as avg_yearly +from + lineitem, + part +where + p_partkey = l_partkey + and p_brand = 'Brand#23' + and p_container = 'MED BOX' + and l_quantity < ( + select + 0.2 * avg(l_quantity) + from + lineitem + where + l_partkey = p_partkey +); \ No newline at end of file diff --git a/rust/ballista/rust/benchmarks/tpch/queries/q18.sql b/rust/ballista/rust/benchmarks/tpch/queries/q18.sql new file mode 100644 index 0000000000000..835de28a57be2 --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/queries/q18.sql @@ -0,0 +1,32 @@ +select + c_name, + c_custkey, + o_orderkey, + o_orderdate, + o_totalprice, + sum(l_quantity) +from + customer, + orders, + lineitem +where + o_orderkey in ( + select + l_orderkey + from + lineitem + group by + l_orderkey having + sum(l_quantity) > 300 + ) + and c_custkey = o_custkey + and o_orderkey = l_orderkey +group by + c_name, + c_custkey, + o_orderkey, + o_orderdate, + o_totalprice +order by + o_totalprice desc, + o_orderdate; \ No newline at end of file diff --git a/rust/ballista/rust/benchmarks/tpch/queries/q19.sql b/rust/ballista/rust/benchmarks/tpch/queries/q19.sql new file mode 100644 index 0000000000000..56668e73f86f6 --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/queries/q19.sql @@ -0,0 +1,35 @@ +select + sum(l_extendedprice* (1 - l_discount)) as revenue +from + lineitem, + part +where + ( + p_partkey = l_partkey + and p_brand = 'Brand#12' + and p_container in ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG') + and l_quantity >= 1 and l_quantity <= 1 + 10 + and p_size between 1 and 5 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ) + or + ( + p_partkey = l_partkey + and p_brand = 'Brand#23' + and p_container in ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK') + and l_quantity >= 10 and l_quantity <= 10 + 10 + and p_size between 1 and 10 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ) + or + ( + p_partkey = l_partkey + and p_brand = 'Brand#34' + and p_container in ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG') + and l_quantity >= 20 and l_quantity <= 20 + 10 + and p_size between 1 and 15 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ); \ No newline at end of file diff --git a/rust/ballista/rust/benchmarks/tpch/queries/q2.sql b/rust/ballista/rust/benchmarks/tpch/queries/q2.sql new file mode 100644 index 0000000000000..f66af210205e9 --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/queries/q2.sql @@ -0,0 +1,43 @@ +select + s_acctbal, + s_name, + n_name, + p_partkey, + p_mfgr, + s_address, + s_phone, + s_comment +from + part, + supplier, + partsupp, + nation, + region +where + p_partkey = ps_partkey + and s_suppkey = ps_suppkey + and p_size = 15 + and p_type like '%BRASS' + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'EUROPE' + and ps_supplycost = ( + select + min(ps_supplycost) + from + partsupp, + supplier, + nation, + region + where + p_partkey = ps_partkey + and s_suppkey = ps_suppkey + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'EUROPE' +) +order by + s_acctbal desc, + n_name, + s_name, + p_partkey; \ No newline at end of file diff --git a/rust/ballista/rust/benchmarks/tpch/queries/q20.sql b/rust/ballista/rust/benchmarks/tpch/queries/q20.sql new file mode 100644 index 0000000000000..f0339a6013c24 --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/queries/q20.sql @@ -0,0 +1,37 @@ +select + s_name, + s_address +from + supplier, + nation +where + s_suppkey in ( + select + ps_suppkey + from + partsupp + where + ps_partkey in ( + select + p_partkey + from + part + where + p_name like 'forest%' + ) + and ps_availqty > ( + select + 0.5 * sum(l_quantity) + from + lineitem + where + l_partkey = ps_partkey + and l_suppkey = ps_suppkey + and l_shipdate >= date '1994-01-01' + and l_shipdate < 'date 1994-01-01' + interval '1' year + ) + ) + and s_nationkey = n_nationkey + and n_name = 'CANADA' +order by + s_name; \ No newline at end of file diff --git a/rust/ballista/rust/benchmarks/tpch/queries/q21.sql b/rust/ballista/rust/benchmarks/tpch/queries/q21.sql new file mode 100644 index 0000000000000..9d2fe32cee228 --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/queries/q21.sql @@ -0,0 +1,39 @@ +select + s_name, + count(*) as numwait +from + supplier, + lineitem l1, + orders, + nation +where + s_suppkey = l1.l_suppkey + and o_orderkey = l1.l_orderkey + and o_orderstatus = 'F' + and l1.l_receiptdate > l1.l_commitdate + and exists ( + select + * + from + lineitem l2 + where + l2.l_orderkey = l1.l_orderkey + and l2.l_suppkey <> l1.l_suppkey + ) + and not exists ( + select + * + from + lineitem l3 + where + l3.l_orderkey = l1.l_orderkey + and l3.l_suppkey <> l1.l_suppkey + and l3.l_receiptdate > l3.l_commitdate + ) + and s_nationkey = n_nationkey + and n_name = 'SAUDI ARABIA' +group by + s_name +order by + numwait desc, + s_name; \ No newline at end of file diff --git a/rust/ballista/rust/benchmarks/tpch/queries/q22.sql b/rust/ballista/rust/benchmarks/tpch/queries/q22.sql new file mode 100644 index 0000000000000..90aea6fd74f5c --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/queries/q22.sql @@ -0,0 +1,37 @@ +select + cntrycode, + count(*) as numcust, + sum(c_acctbal) as totacctbal +from + ( + select + substring(c_phone from 1 for 2) as cntrycode, + c_acctbal + from + customer + where + substring(c_phone from 1 for 2) in + ('13', '31', '23', '29', '30', '18', '17') + and c_acctbal > ( + select + avg(c_acctbal) + from + customer + where + c_acctbal > 0.00 + and substring(c_phone from 1 for 2) in + ('13', '31', '23', '29', '30', '18', '17') + ) + and not exists ( + select + * + from + orders + where + o_custkey = c_custkey + ) + ) as custsale +group by + cntrycode +order by + cntrycode; \ No newline at end of file diff --git a/rust/ballista/rust/benchmarks/tpch/queries/q3.sql b/rust/ballista/rust/benchmarks/tpch/queries/q3.sql new file mode 100644 index 0000000000000..7dbc6d9ef6783 --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/queries/q3.sql @@ -0,0 +1,22 @@ +select + l_orderkey, + sum(l_extendedprice * (1 - l_discount)) as revenue, + o_orderdate, + o_shippriority +from + customer, + orders, + lineitem +where + c_mktsegment = 'BUILDING' + and c_custkey = o_custkey + and l_orderkey = o_orderkey + and o_orderdate < date '1995-03-15' + and l_shipdate > date '1995-03-15' +group by + l_orderkey, + o_orderdate, + o_shippriority +order by + revenue desc, + o_orderdate; \ No newline at end of file diff --git a/rust/ballista/rust/benchmarks/tpch/queries/q4.sql b/rust/ballista/rust/benchmarks/tpch/queries/q4.sql new file mode 100644 index 0000000000000..74a620dbc8a6d --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/queries/q4.sql @@ -0,0 +1,21 @@ +select + o_orderpriority, + count(*) as order_count +from + orders +where + o_orderdate >= '1993-07-01' + and o_orderdate < date '1993-07-01' + interval '3' month + and exists ( + select + * + from + lineitem + where + l_orderkey = o_orderkey + and l_commitdate < l_receiptdate + ) +group by + o_orderpriority +order by + o_orderpriority; \ No newline at end of file diff --git a/rust/ballista/rust/benchmarks/tpch/queries/q5.sql b/rust/ballista/rust/benchmarks/tpch/queries/q5.sql new file mode 100644 index 0000000000000..5a336b231184b --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/queries/q5.sql @@ -0,0 +1,24 @@ +select + n_name, + sum(l_extendedprice * (1 - l_discount)) as revenue +from + customer, + orders, + lineitem, + supplier, + nation, + region +where + c_custkey = o_custkey + and l_orderkey = o_orderkey + and l_suppkey = s_suppkey + and c_nationkey = s_nationkey + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'ASIA' + and o_orderdate >= date '1994-01-01' + and o_orderdate < date '1995-01-01' +group by + n_name +order by + revenue desc; \ No newline at end of file diff --git a/rust/ballista/rust/benchmarks/tpch/queries/q6.sql b/rust/ballista/rust/benchmarks/tpch/queries/q6.sql new file mode 100644 index 0000000000000..5806f980f8088 --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/queries/q6.sql @@ -0,0 +1,9 @@ +select + sum(l_extendedprice * l_discount) as revenue +from + lineitem +where + l_shipdate >= date '1994-01-01' + and l_shipdate < date '1995-01-01' + and l_discount between 0.06 - 0.01 and 0.06 + 0.01 + and l_quantity < 24; \ No newline at end of file diff --git a/rust/ballista/rust/benchmarks/tpch/queries/q7.sql b/rust/ballista/rust/benchmarks/tpch/queries/q7.sql new file mode 100644 index 0000000000000..d53877c8dde68 --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/queries/q7.sql @@ -0,0 +1,39 @@ +select + supp_nation, + cust_nation, + l_year, + sum(volume) as revenue +from + ( + select + n1.n_name as supp_nation, + n2.n_name as cust_nation, + extract(year from l_shipdate) as l_year, + l_extendedprice * (1 - l_discount) as volume + from + supplier, + lineitem, + orders, + customer, + nation n1, + nation n2 + where + s_suppkey = l_suppkey + and o_orderkey = l_orderkey + and c_custkey = o_custkey + and s_nationkey = n1.n_nationkey + and c_nationkey = n2.n_nationkey + and ( + (n1.n_name = 'FRANCE' and n2.n_name = 'GERMANY') + or (n1.n_name = 'GERMANY' and n2.n_name = 'FRANCE') + ) + and l_shipdate between date '1995-01-01' and date '1996-12-31' + ) as shipping +group by + supp_nation, + cust_nation, + l_year +order by + supp_nation, + cust_nation, + l_year; \ No newline at end of file diff --git a/rust/ballista/rust/benchmarks/tpch/queries/q8.sql b/rust/ballista/rust/benchmarks/tpch/queries/q8.sql new file mode 100644 index 0000000000000..6ddb2a6747589 --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/queries/q8.sql @@ -0,0 +1,37 @@ +select + o_year, + sum(case + when nation = 'BRAZIL' then volume + else 0 + end) / sum(volume) as mkt_share +from + ( + select + extract(year from o_orderdate) as o_year, + l_extendedprice * (1 - l_discount) as volume, + n2.n_name as nation + from + part, + supplier, + lineitem, + orders, + customer, + nation n1, + nation n2, + region + where + p_partkey = l_partkey + and s_suppkey = l_suppkey + and l_orderkey = o_orderkey + and o_custkey = c_custkey + and c_nationkey = n1.n_nationkey + and n1.n_regionkey = r_regionkey + and r_name = 'AMERICA' + and s_nationkey = n2.n_nationkey + and o_orderdate between date '1995-01-01' and date '1996-12-31' + and p_type = 'ECONOMY ANODIZED STEEL' + ) as all_nations +group by + o_year +order by + o_year; \ No newline at end of file diff --git a/rust/ballista/rust/benchmarks/tpch/queries/q9.sql b/rust/ballista/rust/benchmarks/tpch/queries/q9.sql new file mode 100644 index 0000000000000..587bbc8a207ff --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/queries/q9.sql @@ -0,0 +1,32 @@ +select + nation, + o_year, + sum(amount) as sum_profit +from + ( + select + n_name as nation, + extract(year from o_orderdate) as o_year, + l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount + from + part, + supplier, + lineitem, + partsupp, + orders, + nation + where + s_suppkey = l_suppkey + and ps_suppkey = l_suppkey + and ps_partkey = l_partkey + and p_partkey = l_partkey + and o_orderkey = l_orderkey + and s_nationkey = n_nationkey + and p_name like '%green%' + ) as profit +group by + nation, + o_year +order by + nation, + o_year desc; \ No newline at end of file diff --git a/rust/ballista/rust/benchmarks/tpch/run.sh b/rust/ballista/rust/benchmarks/tpch/run.sh new file mode 100755 index 0000000000000..c8a36b6013cd7 --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/run.sh @@ -0,0 +1,25 @@ +#!/bin/bash +# 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. +set -e + +# This bash script is meant to be run inside the docker-compose environment. Check the README for instructions + +for query in 1 3 5 6 10 12 +do + /tpch benchmark --host ballista-scheduler --port 50050 --query $query --path /data --format tbl --iterations 1 --debug +done diff --git a/rust/ballista/rust/benchmarks/tpch/src/main.rs b/rust/ballista/rust/benchmarks/tpch/src/main.rs new file mode 100644 index 0000000000000..1ba46ea1826ad --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/src/main.rs @@ -0,0 +1,360 @@ +// 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. + +//! Benchmark derived from TPC-H. This is not an official TPC-H benchmark. +//! +//! This is a modified version of the DataFusion version of these benchmarks. + +use std::collections::HashMap; +use std::fs; +use std::path::{Path, PathBuf}; +use std::time::Instant; + +use arrow::datatypes::{DataType, Field, Schema}; +use arrow::util::pretty; +use ballista::prelude::*; +use datafusion::prelude::*; +use parquet::basic::Compression; +use parquet::file::properties::WriterProperties; +use structopt::StructOpt; + +#[derive(Debug, StructOpt)] +struct BenchmarkOpt { + /// Ballista executor host + #[structopt(long = "host")] + host: String, + + /// Ballista executor port + #[structopt(long = "port")] + port: u16, + + /// Query number + #[structopt(long)] + query: usize, + + /// Activate debug mode to see query results + #[structopt(long)] + debug: bool, + + /// Number of iterations of each test run + #[structopt(long = "iterations", default_value = "1")] + iterations: usize, + + /// Batch size when reading CSV or Parquet files + #[structopt(long = "batch-size", default_value = "32768")] + batch_size: usize, + + /// Path to data files + #[structopt(parse(from_os_str), required = true, long = "path")] + path: PathBuf, + + /// File format: `csv`, `tbl` or `parquet` + #[structopt(long = "format")] + file_format: String, +} + +#[derive(Debug, StructOpt)] +struct ConvertOpt { + /// Path to csv files + #[structopt(parse(from_os_str), required = true, short = "i", long = "input")] + input_path: PathBuf, + + /// Output path + #[structopt(parse(from_os_str), required = true, short = "o", long = "output")] + output_path: PathBuf, + + /// Output file format: `csv` or `parquet` + #[structopt(short = "f", long = "format")] + file_format: String, + + /// Compression to use when writing Parquet files + #[structopt(short = "c", long = "compression", default_value = "snappy")] + compression: String, + + /// Number of partitions to produce + #[structopt(short = "p", long = "partitions", default_value = "1")] + partitions: usize, + + /// Batch size when reading CSV or Parquet files + #[structopt(short = "s", long = "batch-size", default_value = "4096")] + batch_size: usize, +} + +#[derive(Debug, StructOpt)] +#[structopt(name = "TPC-H", about = "TPC-H Benchmarks.")] +enum TpchOpt { + Benchmark(BenchmarkOpt), + Convert(ConvertOpt), +} + +const TABLES: &[&str] = &[ + "part", "supplier", "partsupp", "customer", "orders", "lineitem", "nation", "region", +]; + +#[tokio::main] +async fn main() -> Result<()> { + env_logger::init(); + match TpchOpt::from_args() { + TpchOpt::Benchmark(opt) => benchmark(opt).await.map(|_| ()), + TpchOpt::Convert(opt) => convert_tbl(opt).await, + } +} + +async fn benchmark(opt: BenchmarkOpt) -> Result<()> { + println!("Running benchmarks with the following options: {:?}", opt); + + let mut settings = HashMap::new(); + settings.insert("batch.size".to_owned(), format!("{}", opt.batch_size)); + + let ctx = BallistaContext::remote(opt.host.as_str(), opt.port, settings); + + // register tables with Ballista context + let path = opt.path.to_str().unwrap(); + let file_format = opt.file_format.as_str(); + for table in TABLES { + match file_format { + // dbgen creates .tbl ('|' delimited) files without header + "tbl" => { + let path = format!("{}/{}.tbl", path, table); + let schema = get_schema(table); + let options = CsvReadOptions::new() + .schema(&schema) + .delimiter(b'|') + .has_header(false) + .file_extension(".tbl"); + ctx.register_csv(table, &path, options)?; + } + "csv" => { + let path = format!("{}/{}", path, table); + let schema = get_schema(table); + let options = CsvReadOptions::new().schema(&schema).has_header(true); + ctx.register_csv(table, &path, options)?; + } + "parquet" => { + let path = format!("{}/{}", path, table); + ctx.register_parquet(table, &path)?; + } + other => { + unimplemented!("Invalid file format '{}'", other); + } + } + } + + let mut millis = vec![]; + + // run benchmark + let sql = get_query_sql(opt.query)?; + println!("Running benchmark with query {}:\n {}", opt.query, sql); + for i in 0..opt.iterations { + let start = Instant::now(); + let df = ctx.sql(&sql)?; + let mut batches = vec![]; + let mut stream = df.collect().await?; + while let Some(result) = stream.next().await { + let batch = result?; + batches.push(batch); + } + let elapsed = start.elapsed().as_secs_f64() * 1000.0; + millis.push(elapsed as f64); + println!("Query {} iteration {} took {:.1} ms", opt.query, i, elapsed); + if opt.debug { + pretty::print_batches(&batches)?; + } + } + + let avg = millis.iter().sum::() / millis.len() as f64; + println!("Query {} avg time: {:.2} ms", opt.query, avg); + + Ok(()) +} + +fn get_query_sql(query: usize) -> Result { + if query > 0 && query < 23 { + let filename = format!("queries/q{}.sql", query); + Ok(fs::read_to_string(&filename).expect("failed to read query")) + } else { + Err(BallistaError::General( + "invalid query. Expected value between 1 and 22".to_owned(), + )) + } +} + +async fn convert_tbl(opt: ConvertOpt) -> Result<()> { + let output_root_path = Path::new(&opt.output_path); + for table in TABLES { + let start = Instant::now(); + let schema = get_schema(table); + + let input_path = format!("{}/{}.tbl", opt.input_path.to_str().unwrap(), table); + let options = CsvReadOptions::new() + .schema(&schema) + .delimiter(b'|') + .file_extension(".tbl"); + + let config = ExecutionConfig::new().with_batch_size(opt.batch_size); + let mut ctx = ExecutionContext::with_config(config); + + // build plan to read the TBL file + let mut csv = ctx.read_csv(&input_path, options)?; + + // optionally, repartition the file + if opt.partitions > 1 { + csv = csv.repartition(Partitioning::RoundRobinBatch(opt.partitions))? + } + + // create the physical plan + let csv = csv.to_logical_plan(); + let csv = ctx.optimize(&csv)?; + let csv = ctx.create_physical_plan(&csv)?; + + let output_path = output_root_path.join(table); + let output_path = output_path.to_str().unwrap().to_owned(); + + println!( + "Converting '{}' to {} files in directory '{}'", + &input_path, &opt.file_format, &output_path + ); + match opt.file_format.as_str() { + "csv" => ctx.write_csv(csv, output_path).await?, + "parquet" => { + let compression = match opt.compression.as_str() { + "none" => Compression::UNCOMPRESSED, + "snappy" => Compression::SNAPPY, + "brotli" => Compression::BROTLI, + "gzip" => Compression::GZIP, + "lz4" => Compression::LZ4, + "lz0" => Compression::LZO, + "zstd" => Compression::ZSTD, + other => { + return Err(BallistaError::NotImplemented(format!( + "Invalid compression format: {}", + other + ))) + } + }; + let props = WriterProperties::builder() + .set_compression(compression) + .build(); + ctx.write_parquet(csv, output_path, Some(props)).await? + } + other => { + return Err(BallistaError::NotImplemented(format!( + "Invalid output format: {}", + other + ))) + } + } + println!("Conversion completed in {} ms", start.elapsed().as_millis()); + } + + Ok(()) +} + +fn get_schema(table: &str) -> Schema { + // note that the schema intentionally uses signed integers so that any generated Parquet + // files can also be used to benchmark tools that only support signed integers, such as + // Apache Spark + + match table { + "part" => Schema::new(vec![ + Field::new("p_partkey", DataType::Int32, false), + Field::new("p_name", DataType::Utf8, false), + Field::new("p_mfgr", DataType::Utf8, false), + Field::new("p_brand", DataType::Utf8, false), + Field::new("p_type", DataType::Utf8, false), + Field::new("p_size", DataType::Int32, false), + Field::new("p_container", DataType::Utf8, false), + Field::new("p_retailprice", DataType::Float64, false), + Field::new("p_comment", DataType::Utf8, false), + ]), + + "supplier" => Schema::new(vec![ + Field::new("s_suppkey", DataType::Int32, false), + Field::new("s_name", DataType::Utf8, false), + Field::new("s_address", DataType::Utf8, false), + Field::new("s_nationkey", DataType::Int32, false), + Field::new("s_phone", DataType::Utf8, false), + Field::new("s_acctbal", DataType::Float64, false), + Field::new("s_comment", DataType::Utf8, false), + ]), + + "partsupp" => Schema::new(vec![ + Field::new("ps_partkey", DataType::Int32, false), + Field::new("ps_suppkey", DataType::Int32, false), + Field::new("ps_availqty", DataType::Int32, false), + Field::new("ps_supplycost", DataType::Float64, false), + Field::new("ps_comment", DataType::Utf8, false), + ]), + + "customer" => Schema::new(vec![ + Field::new("c_custkey", DataType::Int32, false), + Field::new("c_name", DataType::Utf8, false), + Field::new("c_address", DataType::Utf8, false), + Field::new("c_nationkey", DataType::Int32, false), + Field::new("c_phone", DataType::Utf8, false), + Field::new("c_acctbal", DataType::Float64, false), + Field::new("c_mktsegment", DataType::Utf8, false), + Field::new("c_comment", DataType::Utf8, false), + ]), + + "orders" => Schema::new(vec![ + Field::new("o_orderkey", DataType::Int32, false), + Field::new("o_custkey", DataType::Int32, false), + Field::new("o_orderstatus", DataType::Utf8, false), + Field::new("o_totalprice", DataType::Float64, false), + Field::new("o_orderdate", DataType::Date32, false), + Field::new("o_orderpriority", DataType::Utf8, false), + Field::new("o_clerk", DataType::Utf8, false), + Field::new("o_shippriority", DataType::Int32, false), + Field::new("o_comment", DataType::Utf8, false), + ]), + + "lineitem" => Schema::new(vec![ + Field::new("l_orderkey", DataType::Int32, false), + Field::new("l_partkey", DataType::Int32, false), + Field::new("l_suppkey", DataType::Int32, false), + Field::new("l_linenumber", DataType::Int32, false), + Field::new("l_quantity", DataType::Float64, false), + Field::new("l_extendedprice", DataType::Float64, false), + Field::new("l_discount", DataType::Float64, false), + Field::new("l_tax", DataType::Float64, false), + Field::new("l_returnflag", DataType::Utf8, false), + Field::new("l_linestatus", DataType::Utf8, false), + Field::new("l_shipdate", DataType::Date32, false), + Field::new("l_commitdate", DataType::Date32, false), + Field::new("l_receiptdate", DataType::Date32, false), + Field::new("l_shipinstruct", DataType::Utf8, false), + Field::new("l_shipmode", DataType::Utf8, false), + Field::new("l_comment", DataType::Utf8, false), + ]), + + "nation" => Schema::new(vec![ + Field::new("n_nationkey", DataType::Int32, false), + Field::new("n_name", DataType::Utf8, false), + Field::new("n_regionkey", DataType::Int32, false), + Field::new("n_comment", DataType::Utf8, false), + ]), + + "region" => Schema::new(vec![ + Field::new("r_regionkey", DataType::Int32, false), + Field::new("r_name", DataType::Utf8, false), + Field::new("r_comment", DataType::Utf8, false), + ]), + + _ => unimplemented!(), + } +} diff --git a/rust/ballista/rust/benchmarks/tpch/tpch-gen.sh b/rust/ballista/rust/benchmarks/tpch/tpch-gen.sh new file mode 100755 index 0000000000000..f5147f55f2f69 --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/tpch-gen.sh @@ -0,0 +1,33 @@ +#!/bin/bash +# 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_VERSION=0.4.2-SNAPSHOT + +#set -e + +docker build -t ballistacompute/ballista-tpchgen:$BALLISTA_VERSION -f tpchgen.dockerfile . + +# Generate data into the ./data directory if it does not already exist +FILE=./data/supplier.tbl +if test -f "$FILE"; then + echo "$FILE exists." +else + mkdir data 2>/dev/null + docker run -v `pwd`/data:/data -it --rm ballistacompute/ballista-tpchgen:$BALLISTA_VERSION + ls -l data +fi \ No newline at end of file diff --git a/rust/ballista/rust/benchmarks/tpch/tpchgen.dockerfile b/rust/ballista/rust/benchmarks/tpch/tpchgen.dockerfile new file mode 100644 index 0000000000000..7fc2e5005a52e --- /dev/null +++ b/rust/ballista/rust/benchmarks/tpch/tpchgen.dockerfile @@ -0,0 +1,32 @@ +# 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. + +FROM ubuntu + +RUN apt-get update && \ + apt-get install -y git build-essential + +RUN git clone https://github.com/databricks/tpch-dbgen.git && \ + cd tpch-dbgen && \ + make + +WORKDIR /tpch-dbgen +ADD entrypoint.sh /tpch-dbgen/ + +VOLUME data + +ENTRYPOINT [ "bash", "./entrypoint.sh" ] diff --git a/rust/ballista/rust/client/Cargo.toml b/rust/ballista/rust/client/Cargo.toml new file mode 100644 index 0000000000000..966e2dcbb31f5 --- /dev/null +++ b/rust/ballista/rust/client/Cargo.toml @@ -0,0 +1,34 @@ +# 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. + +[package] +name = "ballista" +description = "Ballista Distributed Compute" +license = "Apache-2.0" +version = "0.4.2-SNAPSHOT" +homepage = "https://github.com/apache/arrow" +repository = "https://github.com/apache/arrow" +authors = ["Apache Arrow "] +edition = "2018" + +[dependencies] +ballista-core = { path = "../core" } +futures = "0.3" +log = "0.4" +tokio = "1.0" +arrow = { git = "https://github.com/apache/arrow", rev="46161d2" } +datafusion = { git = "https://github.com/apache/arrow", rev="46161d2" } diff --git a/rust/ballista/rust/client/README.md b/rust/ballista/rust/client/README.md new file mode 100644 index 0000000000000..00bf3ea5ec6c2 --- /dev/null +++ b/rust/ballista/rust/client/README.md @@ -0,0 +1,22 @@ + + +# Ballista - Rust +This crate contains the Ballista client library. For an example usage, please refer [here](../benchmarks/tpch/README.md). + diff --git a/rust/ballista/rust/client/src/columnar_batch.rs b/rust/ballista/rust/client/src/columnar_batch.rs new file mode 100644 index 0000000000000..d3ff8861faac8 --- /dev/null +++ b/rust/ballista/rust/client/src/columnar_batch.rs @@ -0,0 +1,167 @@ +// 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. + +use std::{collections::HashMap, sync::Arc}; + +use ballista_core::error::{ballista_error, Result}; + +use arrow::{ + array::ArrayRef, + datatypes::{DataType, Schema}, + record_batch::RecordBatch, +}; +use datafusion::scalar::ScalarValue; + +pub type MaybeColumnarBatch = Result>; + +/// Batch of columnar data. +#[allow(dead_code)] +#[derive(Debug, Clone)] + +pub struct ColumnarBatch { + schema: Arc, + columns: HashMap, +} + +impl ColumnarBatch { + pub fn from_arrow(batch: &RecordBatch) -> Self { + let columns = batch + .columns() + .iter() + .enumerate() + .map(|(i, array)| { + ( + batch.schema().field(i).name().clone(), + ColumnarValue::Columnar(array.clone()), + ) + }) + .collect(); + + Self { + schema: batch.schema(), + columns, + } + } + + pub fn from_values(values: &[ColumnarValue], schema: &Schema) -> Self { + let columns = schema + .fields() + .iter() + .enumerate() + .map(|(i, f)| (f.name().clone(), values[i].clone())) + .collect(); + + Self { + schema: Arc::new(schema.clone()), + columns, + } + } + + pub fn to_arrow(&self) -> Result { + let arrays = self + .schema + .fields() + .iter() + .map(|c| { + match self.column(c.name())? { + ColumnarValue::Columnar(array) => Ok(array.clone()), + ColumnarValue::Scalar(_, _) => { + // note that this can be implemented easily if needed + Err(ballista_error("Cannot convert scalar value to Arrow array")) + } + } + }) + .collect::>>()?; + + Ok(RecordBatch::try_new(self.schema.clone(), arrays)?) + } + + pub fn schema(&self) -> Arc { + self.schema.clone() + } + + pub fn num_columns(&self) -> usize { + self.columns.len() + } + + pub fn num_rows(&self) -> usize { + self.columns[self.schema.field(0).name()].len() + } + + pub fn column(&self, name: &str) -> Result<&ColumnarValue> { + Ok(&self.columns[name]) + } + + pub fn memory_size(&self) -> usize { + self.columns.values().map(|c| c.memory_size()).sum() + } +} + +/// A columnar value can either be a scalar value or an Arrow array. +#[allow(dead_code)] +#[derive(Debug, Clone)] + +pub enum ColumnarValue { + Scalar(ScalarValue, usize), + Columnar(ArrayRef), +} + +impl ColumnarValue { + pub fn len(&self) -> usize { + match self { + ColumnarValue::Scalar(_, n) => *n, + ColumnarValue::Columnar(array) => array.len(), + } + } + + pub fn is_empty(&self) -> bool { + self.len() == 0 + } + + pub fn data_type(&self) -> &DataType { + match self { + ColumnarValue::Columnar(array) => array.data_type(), + ColumnarValue::Scalar(value, _) => match value { + ScalarValue::UInt8(_) => &DataType::UInt8, + ScalarValue::UInt16(_) => &DataType::UInt16, + ScalarValue::UInt32(_) => &DataType::UInt32, + ScalarValue::UInt64(_) => &DataType::UInt64, + ScalarValue::Int8(_) => &DataType::Int8, + ScalarValue::Int16(_) => &DataType::Int16, + ScalarValue::Int32(_) => &DataType::Int32, + ScalarValue::Int64(_) => &DataType::Int64, + ScalarValue::Float32(_) => &DataType::Float32, + ScalarValue::Float64(_) => &DataType::Float64, + _ => unimplemented!(), + }, + } + } + + pub fn to_arrow(&self) -> ArrayRef { + match self { + ColumnarValue::Columnar(array) => array.clone(), + ColumnarValue::Scalar(value, n) => value.to_array_of_size(*n), + } + } + + pub fn memory_size(&self) -> usize { + match self { + ColumnarValue::Columnar(array) => array.get_array_memory_size(), + _ => 0, + } + } +} diff --git a/rust/ballista/rust/client/src/context.rs b/rust/ballista/rust/client/src/context.rs new file mode 100644 index 0000000000000..09fe8a0ebfcec --- /dev/null +++ b/rust/ballista/rust/client/src/context.rs @@ -0,0 +1,372 @@ +// 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. + +//! Distributed execution context. + +use std::path::PathBuf; +use std::pin::Pin; +use std::sync::{Arc, Mutex}; +use std::{collections::HashMap, convert::TryInto}; +use std::{fs, time::Duration}; + +use ballista_core::serde::protobuf::scheduler_grpc_client::SchedulerGrpcClient; +use ballista_core::serde::protobuf::{ + execute_query_params::Query, job_status, ExecuteQueryParams, GetJobStatusParams, + GetJobStatusResult, +}; +use ballista_core::{ + client::BallistaClient, + datasource::DFTableAdapter, + error::{BallistaError, Result}, + memory_stream::MemoryStream, +}; + +use arrow::datatypes::Schema; +use datafusion::execution::context::ExecutionContext; +use datafusion::logical_plan::{DFSchema, Expr, LogicalPlan, Partitioning}; +use datafusion::physical_plan::csv::CsvReadOptions; +use datafusion::{dataframe::DataFrame, physical_plan::RecordBatchStream}; +use log::{error, info}; + +#[allow(dead_code)] +struct BallistaContextState { + /// Scheduler host + scheduler_host: String, + /// Scheduler port + scheduler_port: u16, + /// Tables that have been registered with this context + tables: HashMap, + /// General purpose settings + settings: HashMap, +} + +impl BallistaContextState { + pub fn new( + scheduler_host: String, + scheduler_port: u16, + settings: HashMap, + ) -> Self { + Self { + scheduler_host, + scheduler_port, + tables: HashMap::new(), + settings, + } + } +} + +#[allow(dead_code)] + +pub struct BallistaContext { + state: Arc>, +} + +impl BallistaContext { + /// Create a context for executing queries against a remote Ballista scheduler instance + pub fn remote(host: &str, port: u16, settings: HashMap) -> Self { + let state = BallistaContextState::new(host.to_owned(), port, settings); + + Self { + state: Arc::new(Mutex::new(state)), + } + } + + /// Create a DataFrame representing a Parquet table scan + + pub fn read_parquet(&self, path: &str) -> Result { + // convert to absolute path because the executor likely has a different working directory + let path = PathBuf::from(path); + let path = fs::canonicalize(&path)?; + + // use local DataFusion context for now but later this might call the scheduler + let mut ctx = ExecutionContext::new(); + let df = ctx.read_parquet(path.to_str().unwrap())?; + Ok(BallistaDataFrame::from(self.state.clone(), df)) + } + + /// Create a DataFrame representing a CSV table scan + + pub fn read_csv(&self, path: &str, options: CsvReadOptions) -> Result { + // convert to absolute path because the executor likely has a different working directory + let path = PathBuf::from(path); + let path = fs::canonicalize(&path)?; + + // use local DataFusion context for now but later this might call the scheduler + let mut ctx = ExecutionContext::new(); + let df = ctx.read_csv(path.to_str().unwrap(), options)?; + Ok(BallistaDataFrame::from(self.state.clone(), df)) + } + + /// Register a DataFrame as a table that can be referenced from a SQL query + pub fn register_table(&self, name: &str, table: &BallistaDataFrame) -> Result<()> { + let mut state = self.state.lock().unwrap(); + state + .tables + .insert(name.to_owned(), table.to_logical_plan()); + Ok(()) + } + + pub fn register_csv(&self, name: &str, path: &str, options: CsvReadOptions) -> Result<()> { + let df = self.read_csv(path, options)?; + self.register_table(name, &df) + } + + pub fn register_parquet(&self, name: &str, path: &str) -> Result<()> { + let df = self.read_parquet(path)?; + self.register_table(name, &df) + } + + /// Create a DataFrame from a SQL statement + pub fn sql(&self, sql: &str) -> Result { + // use local DataFusion context for now but later this might call the scheduler + let mut ctx = ExecutionContext::new(); + // register tables + let state = self.state.lock().unwrap(); + for (name, plan) in &state.tables { + let plan = ctx.optimize(plan)?; + let execution_plan = ctx.create_physical_plan(&plan)?; + ctx.register_table(name, Arc::new(DFTableAdapter::new(plan, execution_plan))); + } + let df = ctx.sql(sql)?; + Ok(BallistaDataFrame::from(self.state.clone(), df)) + } +} + +/// The Ballista DataFrame is a wrapper around the DataFusion DataFrame and overrides the +/// `collect` method so that the query is executed against Ballista and not DataFusion. + +pub struct BallistaDataFrame { + /// Ballista context state + state: Arc>, + /// DataFusion DataFrame representing logical query plan + df: Arc, +} + +impl BallistaDataFrame { + fn from(state: Arc>, df: Arc) -> Self { + Self { state, df } + } + + pub async fn collect(&self) -> Result>> { + let scheduler_url = { + let state = self.state.lock().unwrap(); + + format!("http://{}:{}", state.scheduler_host, state.scheduler_port) + }; + + info!("Connecting to Ballista scheduler at {}", scheduler_url); + + let mut scheduler = SchedulerGrpcClient::connect(scheduler_url).await?; + + let plan = self.df.to_logical_plan(); + let schema: Schema = plan.schema().as_ref().clone().into(); + + let job_id = scheduler + .execute_query(ExecuteQueryParams { + query: Some(Query::LogicalPlan((&plan).try_into()?)), + }) + .await? + .into_inner() + .job_id; + + loop { + let GetJobStatusResult { status } = scheduler + .get_job_status(GetJobStatusParams { + job_id: job_id.clone(), + }) + .await? + .into_inner(); + let status = status.and_then(|s| s.status).ok_or_else(|| { + BallistaError::Internal("Received empty status message".to_owned()) + })?; + let wait_future = tokio::time::sleep(Duration::from_millis(100)); + match status { + job_status::Status::Queued(_) => { + info!("Job {} still queued...", job_id); + wait_future.await; + } + job_status::Status::Running(_) => { + info!("Job {} is running...", job_id); + wait_future.await; + } + job_status::Status::Failed(err) => { + let msg = format!("Job {} failed: {}", job_id, err.error); + error!("{}", msg); + break Err(BallistaError::General(msg)); + } + job_status::Status::Completed(completed) => { + // TODO: use streaming. Probably need to change the signature of fetch_partition to achieve that + let mut result = vec![]; + for location in completed.partition_location { + let metadata = location.executor_meta.ok_or_else(|| { + BallistaError::Internal("Received empty executor metadata".to_owned()) + })?; + let partition_id = location.partition_id.ok_or_else(|| { + BallistaError::Internal("Received empty partition id".to_owned()) + })?; + let mut ballista_client = + BallistaClient::try_new(metadata.host.as_str(), metadata.port as u16) + .await?; + let stream = ballista_client + .fetch_partition( + &partition_id.job_id, + partition_id.stage_id as usize, + partition_id.partition_id as usize, + ) + .await?; + result + .append(&mut datafusion::physical_plan::common::collect(stream).await?); + } + break Ok(Box::pin(MemoryStream::try_new( + result, + Arc::new(schema), + None, + )?)); + } + }; + } + } + + pub fn select_columns(&self, columns: &[&str]) -> Result { + Ok(Self::from( + self.state.clone(), + self.df + .select_columns(columns) + .map_err(BallistaError::from)?, + )) + } + + pub fn select(&self, expr: &[Expr]) -> Result { + Ok(Self::from( + self.state.clone(), + self.df.select(expr).map_err(BallistaError::from)?, + )) + } + + pub fn filter(&self, expr: Expr) -> Result { + Ok(Self::from( + self.state.clone(), + self.df.filter(expr).map_err(BallistaError::from)?, + )) + } + + pub fn aggregate(&self, group_expr: &[Expr], aggr_expr: &[Expr]) -> Result { + Ok(Self::from( + self.state.clone(), + self.df + .aggregate(group_expr, aggr_expr) + .map_err(BallistaError::from)?, + )) + } + + pub fn limit(&self, n: usize) -> Result { + Ok(Self::from( + self.state.clone(), + self.df.limit(n).map_err(BallistaError::from)?, + )) + } + + pub fn sort(&self, expr: &[Expr]) -> Result { + Ok(Self::from( + self.state.clone(), + self.df.sort(expr).map_err(BallistaError::from)?, + )) + } + + // TODO lifetime issue + // pub fn join(&self, right: Arc, join_type: JoinType, left_cols: &[&str], right_cols: &[&str]) -> + // Result { Ok(Self::from(self.state.clone(), self.df.join(right, join_type, &left_cols, + // &right_cols).map_err(BallistaError::from)?)) } + + pub fn repartition(&self, partitioning_scheme: Partitioning) -> Result { + Ok(Self::from( + self.state.clone(), + self.df + .repartition(partitioning_scheme) + .map_err(BallistaError::from)?, + )) + } + + pub fn schema(&self) -> &DFSchema { + self.df.schema() + } + + pub fn to_logical_plan(&self) -> LogicalPlan { + self.df.to_logical_plan() + } + + pub fn explain(&self, verbose: bool) -> Result { + Ok(Self::from( + self.state.clone(), + self.df.explain(verbose).map_err(BallistaError::from)?, + )) + } +} + +// #[async_trait] +// impl ExecutionContext for BallistaContext { +// async fn get_executor_ids(&self) -> Result> { +// match &self.config.discovery_mode { +// DiscoveryMode::Etcd => etcd_get_executors(&self.config.etcd_urls, "default").await, +// DiscoveryMode::Kubernetes => k8s_get_executors("default", "ballista").await, +// DiscoveryMode::Standalone => Err(ballista_error("Standalone mode not implemented yet")), +// } +// } +// +// async fn execute_task( +// &self, +// executor_meta: ExecutorMeta, +// task: ExecutionTask, +// ) -> Result { +// // TODO what is the point of returning this info since it is based on input arg? +// let shuffle_id = ShuffleId::new(task.job_uuid, task.stage_id, task.partition_id); +// +// let _ = execute_action( +// &executor_meta.host, +// executor_meta.port, +// &Action::Execute(task), +// ) +// .await?; +// +// Ok(shuffle_id) +// } +// +// async fn read_shuffle(&self, shuffle_id: &ShuffleId) -> Result> { +// match self.shuffle_locations.get(shuffle_id) { +// Some(executor_meta) => { +// let batches = execute_action( +// &executor_meta.host, +// executor_meta.port, +// &Action::FetchShuffle(*shuffle_id), +// ) +// .await?; +// Ok(batches +// .iter() +// .map(|b| ColumnarBatch::from_arrow(b)) +// .collect()) +// } +// _ => Err(ballista_error(&format!( +// "Failed to resolve executor UUID for shuffle ID {:?}", +// shuffle_id +// ))), +// } +// } +// +// fn config(&self) -> ExecutorConfig { +// self.config.clone() +// } +// } diff --git a/rust/ballista/rust/client/src/lib.rs b/rust/ballista/rust/client/src/lib.rs new file mode 100644 index 0000000000000..c3c62918680d3 --- /dev/null +++ b/rust/ballista/rust/client/src/lib.rs @@ -0,0 +1,20 @@ +// 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. + +pub mod columnar_batch; +pub mod context; +pub mod prelude; diff --git a/rust/ballista/rust/client/src/prelude.rs b/rust/ballista/rust/client/src/prelude.rs new file mode 100644 index 0000000000000..2f940aef4c976 --- /dev/null +++ b/rust/ballista/rust/client/src/prelude.rs @@ -0,0 +1,23 @@ +// 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 Prelude (common imports) + +pub use crate::context::BallistaContext; +pub use ballista_core::error::{BallistaError, Result}; + +pub use futures::StreamExt; diff --git a/rust/ballista/rust/core/Cargo.toml b/rust/ballista/rust/core/Cargo.toml new file mode 100644 index 0000000000000..b6301918a1ff6 --- /dev/null +++ b/rust/ballista/rust/core/Cargo.toml @@ -0,0 +1,49 @@ +# 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. + +[package] +name = "ballista-core" +description = "Ballista Distributed Compute" +license = "Apache-2.0" +version = "0.4.2-SNAPSHOT" +homepage = "https://github.com/apache/arrow" +repository = "https://github.com/apache/arrow" +authors = ["Apache Arrow "] +edition = "2018" +build = "build.rs" + +[features] +simd = ["datafusion/simd"] + +[dependencies] +async-trait = "0.1.36" +futures = "0.3" +log = "0.4" +prost = "0.7" +sqlparser = "0.8" +tokio = "1.0" +tonic = "0.4" +uuid = { version = "0.8", features = ["v4"] } +arrow = { git = "https://github.com/apache/arrow", rev="46161d2" } +arrow-flight = { git = "https://github.com/apache/arrow", rev="46161d2" } +datafusion = { git = "https://github.com/apache/arrow", rev="46161d2" } + + +[dev-dependencies] + +[build-dependencies] +tonic-build = { version = "0.4" } diff --git a/rust/ballista/rust/core/README.md b/rust/ballista/rust/core/README.md new file mode 100644 index 0000000000000..f97952b3f7023 --- /dev/null +++ b/rust/ballista/rust/core/README.md @@ -0,0 +1,21 @@ + + +# Ballista - Rust +This crate contains the core Ballista types. diff --git a/rust/ballista/rust/core/build.rs b/rust/ballista/rust/core/build.rs new file mode 100644 index 0000000000000..6ad153e87c888 --- /dev/null +++ b/rust/ballista/rust/core/build.rs @@ -0,0 +1,26 @@ +// 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. + +fn main() -> Result<(), String> { + // for use in docker build where file changes can be wonky + println!("cargo:rerun-if-env-changed=FORCE_REBUILD"); + + println!("cargo:rerun-if-changed=proto/ballista.proto"); + tonic_build::configure() + .compile(&["proto/ballista.proto"], &["proto"]) + .map_err(|e| format!("protobuf compilation failed: {}", e)) +} diff --git a/rust/ballista/rust/core/proto/ballista.proto b/rust/ballista/rust/core/proto/ballista.proto new file mode 100644 index 0000000000000..ff0727b78875e --- /dev/null +++ b/rust/ballista/rust/core/proto/ballista.proto @@ -0,0 +1,818 @@ +/* + * 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. + */ + +syntax = "proto3"; + +package ballista.protobuf; + +option java_multiple_files = true; +option java_package = "org.ballistacompute.protobuf"; +option java_outer_classname = "BallistaProto"; + +/////////////////////////////////////////////////////////////////////////////////////////////////// +// Ballista Logical Plan +/////////////////////////////////////////////////////////////////////////////////////////////////// + +// logical expressions +message LogicalExprNode { + oneof ExprType { + // column references + string column_name = 1; + + // alias + AliasNode alias = 2; + + ScalarValue literal = 3; + + + // binary expressions + BinaryExprNode binary_expr = 4; + + // aggregate expressions + AggregateExprNode aggregate_expr = 5; + + // null checks + IsNull is_null_expr = 6; + IsNotNull is_not_null_expr = 7; + Not not_expr = 8; + + BetweenNode between = 9; + CaseNode case_ = 10; + CastNode cast = 11; + SortExprNode sort = 12; + NegativeNode negative = 13; + InListNode in_list = 14; + bool wildcard = 15; + ScalarFunctionNode scalar_function = 16; + } +} + +message IsNull { + LogicalExprNode expr = 1; +} + +message IsNotNull { + LogicalExprNode expr = 1; +} + +message Not { + LogicalExprNode expr = 1; +} + +message AliasNode { + LogicalExprNode expr = 1; + string alias = 2; +} + +message BinaryExprNode { + LogicalExprNode l = 1; + LogicalExprNode r = 2; + string op = 3; +} + +message NegativeNode { + LogicalExprNode expr = 1; +} + +message InListNode { + LogicalExprNode expr = 1; + repeated LogicalExprNode list = 2; + bool negated = 3; +} + +enum ScalarFunction { + SQRT = 0; + SIN = 1; + COS = 2; + TAN = 3; + ASIN = 4; + ACOS = 5; + ATAN = 6; + EXP = 7; + LOG = 8; + LOG2 = 9; + LOG10 = 10; + FLOOR = 11; + CEIL = 12; + ROUND = 13; + TRUNC = 14; + ABS = 15; + SIGNUM = 16; + OCTETLENGTH = 17; + CONCAT = 18; + LOWER = 19; + UPPER = 20; + TRIM = 21; + LTRIM = 22; + RTRIM = 23; + TOTIMESTAMP = 24; + ARRAY = 25; + NULLIF = 26; + DATETRUNC = 27; + MD5 = 28; + SHA224 = 29; + SHA256 = 30; + SHA384 = 31; + SHA512 = 32; +} + +message ScalarFunctionNode { + ScalarFunction fun = 1; + repeated LogicalExprNode expr = 2; +} + +enum AggregateFunction { + MIN = 0; + MAX = 1; + SUM = 2; + AVG = 3; + COUNT = 4; +} + +message AggregateExprNode { + AggregateFunction aggr_function = 1; + LogicalExprNode expr = 2; +} + +message BetweenNode { + LogicalExprNode expr = 1; + bool negated = 2; + LogicalExprNode low = 3; + LogicalExprNode high = 4; +} + +message CaseNode { + LogicalExprNode expr = 1; + repeated WhenThen when_then_expr = 2; + LogicalExprNode else_expr = 3; +} + +message WhenThen { + LogicalExprNode when_expr = 1; + LogicalExprNode then_expr = 2; +} + +message CastNode { + LogicalExprNode expr = 1; + ArrowType arrow_type = 2; +} + +message SortExprNode { + LogicalExprNode expr = 1; + bool asc = 2; + bool nulls_first = 3; +} + +// LogicalPlan is a nested type +message LogicalPlanNode { + oneof LogicalPlanType { + CsvTableScanNode csv_scan = 1; + ParquetTableScanNode parquet_scan = 2; + ProjectionNode projection = 3; + SelectionNode selection = 4; + LimitNode limit = 5; + AggregateNode aggregate = 6; + JoinNode join = 7; + SortNode sort = 8; + RepartitionNode repartition = 9; + EmptyRelationNode empty_relation = 10; + CreateExternalTableNode create_external_table = 11; + ExplainNode explain = 12; + } +} + +message ProjectionColumns { + repeated string columns = 1; +} + +message CsvTableScanNode { + string table_name = 1; + string path = 2; + bool has_header = 3; + string delimiter = 4; + string file_extension = 5; + ProjectionColumns projection = 6; + Schema schema = 7; + repeated LogicalExprNode filters = 8; +} + +message ParquetTableScanNode { + string table_name = 1; + string path = 2; + ProjectionColumns projection = 3; + Schema schema = 4; + repeated LogicalExprNode filters = 5; +} + +message ProjectionNode { + LogicalPlanNode input = 1; + repeated LogicalExprNode expr = 2; +} + +message SelectionNode { + LogicalPlanNode input = 1; + LogicalExprNode expr = 2; +} + +message SortNode{ + LogicalPlanNode input = 1; + repeated LogicalExprNode expr = 2; +} + +message RepartitionNode{ + LogicalPlanNode input = 1; + oneof partition_method { + uint64 round_robin = 2; + HashRepartition hash = 3; + } +} + +message HashRepartition { + repeated LogicalExprNode hash_expr = 1; + uint64 partition_count = 2; +} + +message EmptyRelationNode{ + bool produce_one_row = 1; +} + +message CreateExternalTableNode{ + string name = 1; + string location = 2; + FileType file_type = 3; + bool has_header = 4; + Schema schema = 5; +} + +enum FileType{ + NdJson = 0; + Parquet = 1; + CSV = 2; +} + +message ExplainNode{ + LogicalPlanNode input = 1; + bool verbose = 2; +} + +message DfField{ + string qualifier = 2; + Field field = 1; +} + +message AggregateNode { + LogicalPlanNode input = 1; + repeated LogicalExprNode group_expr = 2; + repeated LogicalExprNode aggr_expr = 3; +} + +enum JoinType { + INNER = 0; + LEFT = 1; + RIGHT = 2; +} + +message JoinNode { + LogicalPlanNode left = 1; + LogicalPlanNode right = 2; + JoinType join_type = 3; + repeated string left_join_column = 4; + repeated string right_join_column = 5; +} + +message LimitNode { + LogicalPlanNode input = 1; + uint32 limit = 2; +} + +message SelectionExecNode { + LogicalExprNode expr = 1; +} + +/////////////////////////////////////////////////////////////////////////////////////////////////// +// Ballista Physical Plan +/////////////////////////////////////////////////////////////////////////////////////////////////// + +// PhysicalPlanNode is a nested type +message PhysicalPlanNode { + oneof PhysicalPlanType { + ParquetScanExecNode parquet_scan = 1; + CsvScanExecNode csv_scan = 2; + EmptyExecNode empty = 3; + ProjectionExecNode projection = 4; + GlobalLimitExecNode global_limit = 6; + LocalLimitExecNode local_limit = 7; + HashAggregateExecNode hash_aggregate = 8; + HashJoinExecNode hash_join = 9; + ShuffleReaderExecNode shuffle_reader = 10; + SortExecNode sort = 11; + CoalesceBatchesExecNode coalesce_batches = 12; + FilterExecNode filter = 13; + MergeExecNode merge = 14; + UnresolvedShuffleExecNode unresolved = 15; + RepartitionExecNode repartition = 16; + } +} + +message UnresolvedShuffleExecNode { + repeated uint32 query_stage_ids = 1; + Schema schema = 2; + uint32 partition_count = 3; +} + +message FilterExecNode { + PhysicalPlanNode input = 1; + LogicalExprNode expr = 2; +} + +message ParquetScanExecNode { + repeated string filename = 1; + repeated uint32 projection = 2; + uint32 num_partitions = 3; + uint32 batch_size = 4; +} + +message CsvScanExecNode { + string path = 1; + repeated uint32 projection = 2; + Schema schema = 3; + string file_extension = 4; + bool has_header = 5; + uint32 batch_size = 6; + string delimiter = 7; + + // partition filenames + repeated string filename = 8; +} + +message HashJoinExecNode { + PhysicalPlanNode left = 1; + PhysicalPlanNode right = 2; + repeated JoinOn on = 3; + JoinType join_type = 4; + +} + +message JoinOn { + string left = 1; + string right = 2; +} + + +message EmptyExecNode { + bool produce_one_row = 1; + Schema schema = 2; +} + +message ProjectionExecNode { + PhysicalPlanNode input = 1; + repeated LogicalExprNode expr = 2; + repeated string expr_name = 3; +} + +enum AggregateMode { + PARTIAL = 0; + FINAL = 1; +} + +message HashAggregateExecNode { + repeated LogicalExprNode group_expr = 1; + repeated LogicalExprNode aggr_expr = 2; + AggregateMode mode = 3; + PhysicalPlanNode input = 4; + repeated string group_expr_name = 5; + repeated string aggr_expr_name = 6; + // we need the input schema to the partial aggregate to pass to the final aggregate + Schema input_schema = 7; +} + +message ShuffleReaderExecNode { + repeated PartitionLocation partition_location = 1; + Schema schema = 2; +} + +message GlobalLimitExecNode { + PhysicalPlanNode input = 1; + uint32 limit = 2; +} + +message LocalLimitExecNode { + PhysicalPlanNode input = 1; + uint32 limit = 2; +} + +message SortExecNode { + PhysicalPlanNode input = 1; + repeated LogicalExprNode expr = 2; +} + +message CoalesceBatchesExecNode { + PhysicalPlanNode input = 1; + uint32 target_batch_size = 2; +} + +message MergeExecNode { + PhysicalPlanNode input = 1; +} + +message RepartitionExecNode{ + PhysicalPlanNode input = 1; + oneof partition_method { + uint64 round_robin = 2; + HashRepartition hash = 3; + uint64 unknown = 4; + } +} + +/////////////////////////////////////////////////////////////////////////////////////////////////// +// Ballista Scheduling +/////////////////////////////////////////////////////////////////////////////////////////////////// + +message KeyValuePair { + string key = 1; + string value = 2; +} + +message Action { + + oneof ActionType { + // Execute a logical query plan + LogicalPlanNode query = 1; + + // Execute one partition of a physical query plan + ExecutePartition execute_partition = 2; + + // Fetch a partition from an executor + PartitionId fetch_partition = 3; + } + + // configuration settings + repeated KeyValuePair settings = 100; +} + +message ExecutePartition { + string job_id = 1; + uint32 stage_id = 2; + repeated uint32 partition_id = 3; + PhysicalPlanNode plan = 4; + // The task could need to read partitions from other executors + repeated PartitionLocation partition_location = 5; +} + +// Mapping from partition id to executor id +message PartitionLocation { + PartitionId partition_id = 1; + ExecutorMetadata executor_meta = 2; + PartitionStats partition_stats = 3; +} + +// Unique identifier for a materialized partition of data +message PartitionId { + string job_id = 1; + uint32 stage_id = 2; + uint32 partition_id = 4; +} + +message PartitionStats { + int64 num_rows = 1; + int64 num_batches = 2; + int64 num_bytes = 3; + repeated ColumnStats column_stats = 4; +} + +message ColumnStats { + ScalarValue min_value = 1; + ScalarValue max_value = 2; + uint32 null_count = 3; + uint32 distinct_count = 4; +} + +message ExecutorMetadata { + string id = 1; + string host = 2; + uint32 port = 3; +} + +message GetExecutorMetadataParams {} + +message GetExecutorMetadataResult { + repeated ExecutorMetadata metadata = 1; +} + +message RunningTask { + string executor_id = 1; +} + +message FailedTask { + string error = 1; +} + +message CompletedTask { + string executor_id = 1; +} + +message TaskStatus { + PartitionId partition_id = 1; + oneof status { + RunningTask running = 2; + FailedTask failed = 3; + CompletedTask completed = 4; + } +} + +message PollWorkParams { + ExecutorMetadata metadata = 1; + bool can_accept_task = 2; + // All tasks must be reported until they reach the failed or completed state + repeated TaskStatus task_status = 3; +} + +message TaskDefinition { + PartitionId task_id = 1; + PhysicalPlanNode plan = 2; +} + +message PollWorkResult { + TaskDefinition task = 1; +} + +message ExecuteQueryParams { + oneof query { + LogicalPlanNode logical_plan = 1; + string sql = 2; + }} + +message ExecuteSqlParams { + string sql = 1; +} + +message ExecuteQueryResult { + string job_id = 1; +} + +message GetJobStatusParams { + string job_id = 1; +} + +message CompletedJob { + repeated PartitionLocation partition_location = 1; +} + +message QueuedJob {} + +// TODO: add progress report +message RunningJob {} + +message FailedJob { + string error = 1; +} + +message JobStatus { + oneof status { + QueuedJob queued = 1; + RunningJob running = 2; + FailedJob failed = 3; + CompletedJob completed = 4; + } +} + +message GetJobStatusResult { + JobStatus status = 1; +} + +message GetFileMetadataParams { + string path = 1; + FileType file_type = 2; +} + +message GetFileMetadataResult { + Schema schema = 1; + repeated FilePartitionMetadata partitions = 2; +} + +message FilePartitionMetadata { + repeated string filename = 1; +} + +service SchedulerGrpc { + rpc GetExecutorsMetadata (GetExecutorMetadataParams) returns (GetExecutorMetadataResult) {} + + // Executors must poll the scheduler for heartbeat and to receive tasks + rpc PollWork (PollWorkParams) returns (PollWorkResult) {} + + rpc GetFileMetadata (GetFileMetadataParams) returns (GetFileMetadataResult) {} + + rpc ExecuteQuery (ExecuteQueryParams) returns (ExecuteQueryResult) {} + + rpc GetJobStatus (GetJobStatusParams) returns (GetJobStatusResult) {} +} + +/////////////////////////////////////////////////////////////////////////////////////////////////// +// Arrow Data Types +/////////////////////////////////////////////////////////////////////////////////////////////////// + +message Schema { + repeated Field columns = 1; +} + +message Field { + // name of the field + string name = 1; + ArrowType arrow_type = 2; + bool nullable = 3; + // for complex data types like structs, unions + repeated Field children = 4; +} + +message FixedSizeBinary{ + int32 length = 1; +} + +message Timestamp{ + TimeUnit time_unit = 1; + string timezone = 2; +} + +enum DateUnit{ + Day = 0; + DateMillisecond = 1; +} + +enum TimeUnit{ + Second = 0; + TimeMillisecond = 1; + Microsecond = 2; + Nanosecond = 3; +} + +enum IntervalUnit{ + YearMonth = 0; + DayTime = 1; +} + +message Decimal{ + uint64 whole = 1; + uint64 fractional = 2; +} + +message List{ + Field field_type = 1; +} + +message FixedSizeList{ + Field field_type = 1; + int32 list_size = 2; +} + +message Dictionary{ + ArrowType key = 1; + ArrowType value = 2; +} + +message Struct{ + repeated Field sub_field_types = 1; +} + +message Union{ + repeated Field union_types = 1; +} + + +message ScalarListValue{ + ScalarType datatype = 1; + repeated ScalarValue values = 2; +} + + + +message ScalarValue{ + oneof value{ + bool bool_value = 1; + string utf8_value = 2; + string large_utf8_value = 3; + int32 int8_value = 4; + int32 int16_value = 5; + int32 int32_value = 6; + int64 int64_value = 7; + uint32 uint8_value = 8; + uint32 uint16_value = 9; + uint32 uint32_value = 10; + uint64 uint64_value = 11; + float float32_value = 12; + double float64_value = 13; + //Literal Date32 value always has a unit of day + int32 date_32_value = 14; + int64 time_microsecond_value = 15; + int64 time_nanosecond_value = 16; + ScalarListValue list_value = 17; + ScalarType null_list_value = 18; + + PrimitiveScalarType null_value = 19; + } +} + +// Contains all valid datafusion scalar type except for +// List +enum PrimitiveScalarType{ + + BOOL = 0; // arrow::Type::BOOL + UINT8 = 1; // arrow::Type::UINT8 + INT8 = 2; // arrow::Type::INT8 + UINT16 = 3; // represents arrow::Type fields in src/arrow/type.h + INT16 = 4; + UINT32 = 5; + INT32 = 6; + UINT64 = 7; + INT64 = 8; + FLOAT32 = 9; + FLOAT64 = 10; + UTF8 = 11; + LARGE_UTF8 = 12; + DATE32 = 13; + TIME_MICROSECOND = 14; + TIME_NANOSECOND = 15; + NULL = 16; +} + +message ScalarType{ + oneof datatype{ + PrimitiveScalarType scalar = 1; + ScalarListType list = 2; + } +} + +message ScalarListType{ + repeated string field_names = 3; + PrimitiveScalarType deepest_type = 2; +} + +// Broke out into multiple message types so that type +// metadata did not need to be in separate message +//All types that are of the empty message types contain no additional metadata +// about the type +message ArrowType{ + oneof arrow_type_enum{ + EmptyMessage NONE = 1; // arrow::Type::NA + EmptyMessage BOOL = 2; // arrow::Type::BOOL + EmptyMessage UINT8 = 3; // arrow::Type::UINT8 + EmptyMessage INT8 = 4; // arrow::Type::INT8 + EmptyMessage UINT16 =5; // represents arrow::Type fields in src/arrow/type.h + EmptyMessage INT16 = 6; + EmptyMessage UINT32 =7; + EmptyMessage INT32 = 8; + EmptyMessage UINT64 =9; + EmptyMessage INT64 =10 ; + EmptyMessage FLOAT16 =11 ; + EmptyMessage FLOAT32 =12 ; + EmptyMessage FLOAT64 =13 ; + EmptyMessage UTF8 =14 ; + EmptyMessage LARGE_UTF8 = 32; + EmptyMessage BINARY =15 ; + int32 FIXED_SIZE_BINARY =16 ; + EmptyMessage LARGE_BINARY = 31; + EmptyMessage DATE32 =17 ; + EmptyMessage DATE64 =18 ; + TimeUnit DURATION = 19; + Timestamp TIMESTAMP =20 ; + TimeUnit TIME32 =21 ; + TimeUnit TIME64 =22 ; + IntervalUnit INTERVAL =23 ; + Decimal DECIMAL =24 ; + List LIST =25; + List LARGE_LIST = 26; + FixedSizeList FIXED_SIZE_LIST = 27; + Struct STRUCT =28; + Union UNION =29; + Dictionary DICTIONARY =30; + } +} + + + + + +//Useful for representing an empty enum variant in rust +// E.G. enum example{One, Two(i32)} +// maps to +// message example{ +// oneof{ +// EmptyMessage One = 1; +// i32 Two = 2; +// } +//} +message EmptyMessage{} diff --git a/rust/ballista/rust/core/src/client.rs b/rust/ballista/rust/core/src/client.rs new file mode 100644 index 0000000000000..c882cfc6c2e7e --- /dev/null +++ b/rust/ballista/rust/core/src/client.rs @@ -0,0 +1,211 @@ +// 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. + +//! Client API for sending requests to executors. + +use std::sync::Arc; +use std::{collections::HashMap, pin::Pin}; +use std::{ + convert::{TryFrom, TryInto}, + task::{Context, Poll}, +}; + +use crate::error::{ballista_error, BallistaError, Result}; +use crate::memory_stream::MemoryStream; +use crate::serde::protobuf::{self}; +use crate::serde::scheduler::{ + Action, ExecutePartition, ExecutePartitionResult, PartitionId, PartitionStats, +}; + +use arrow::record_batch::RecordBatch; +use arrow::{ + array::{StringArray, StructArray}, + error::{ArrowError, Result as ArrowResult}, +}; +use arrow::{datatypes::Schema, datatypes::SchemaRef}; +use arrow_flight::utils::flight_data_to_arrow_batch; +use arrow_flight::Ticket; +use arrow_flight::{flight_service_client::FlightServiceClient, FlightData}; +use datafusion::physical_plan::common::collect; +use datafusion::physical_plan::{ExecutionPlan, SendableRecordBatchStream}; +use datafusion::{logical_plan::LogicalPlan, physical_plan::RecordBatchStream}; +use futures::{Stream, StreamExt}; +use log::debug; +use prost::Message; +use tonic::Streaming; +use uuid::Uuid; + +/// Client for interacting with Ballista executors. +#[derive(Clone)] +pub struct BallistaClient { + flight_client: FlightServiceClient, +} + +impl BallistaClient { + /// Create a new BallistaClient to connect to the executor listening on the specified + /// host and port + + pub async fn try_new(host: &str, port: u16) -> Result { + let addr = format!("http://{}:{}", host, port); + debug!("BallistaClient connecting to {}", addr); + let flight_client = FlightServiceClient::connect(addr.clone()) + .await + .map_err(|e| { + BallistaError::General(format!( + "Error connecting to Ballista scheduler or executor at {}: {:?}", + addr, e + )) + })?; + debug!("BallistaClient connected OK"); + + Ok(Self { flight_client }) + } + + /// Execute one partition of a physical query plan against the executor + pub async fn execute_partition( + &mut self, + job_id: String, + stage_id: usize, + partition_id: Vec, + plan: Arc, + ) -> Result> { + let action = Action::ExecutePartition(ExecutePartition { + job_id, + stage_id, + partition_id, + plan, + shuffle_locations: Default::default(), + }); + let stream = self.execute_action(&action).await?; + let batches = collect(stream).await?; + + batches + .iter() + .map(|batch| { + if batch.num_rows() != 1 { + Err(BallistaError::General( + "execute_partition received wrong number of rows".to_owned(), + )) + } else { + let path = batch + .column(0) + .as_any() + .downcast_ref::() + .expect("execute_partition expected column 0 to be a StringArray"); + + let stats = batch + .column(1) + .as_any() + .downcast_ref::() + .expect("execute_partition expected column 1 to be a StructArray"); + + Ok(ExecutePartitionResult::new( + path.value(0), + PartitionStats::from_arrow_struct_array(stats), + )) + } + }) + .collect::>>() + } + + /// Fetch a partition from an executor + pub async fn fetch_partition( + &mut self, + job_id: &str, + stage_id: usize, + partition_id: usize, + ) -> Result { + let action = Action::FetchPartition(PartitionId::new(job_id, stage_id, partition_id)); + self.execute_action(&action).await + } + + /// Execute an action and retrieve the results + pub async fn execute_action(&mut self, action: &Action) -> Result { + let serialized_action: protobuf::Action = action.to_owned().try_into()?; + + let mut buf: Vec = Vec::with_capacity(serialized_action.encoded_len()); + + serialized_action + .encode(&mut buf) + .map_err(|e| BallistaError::General(format!("{:?}", e)))?; + + let request = tonic::Request::new(Ticket { ticket: buf }); + + let mut stream = self + .flight_client + .do_get(request) + .await + .map_err(|e| BallistaError::General(format!("{:?}", e)))? + .into_inner(); + + // the schema should be the first message returned, else client should error + match stream + .message() + .await + .map_err(|e| BallistaError::General(format!("{:?}", e)))? + { + Some(flight_data) => { + // convert FlightData to a stream + let schema = Arc::new(Schema::try_from(&flight_data)?); + + // all the remaining stream messages should be dictionary and record batches + Ok(Box::pin(FlightDataStream::new(stream, schema))) + } + None => Err(ballista_error( + "Did not receive schema batch from flight server", + )), + } + } +} + +struct FlightDataStream { + stream: Streaming, + schema: SchemaRef, +} + +impl FlightDataStream { + pub fn new(stream: Streaming, schema: SchemaRef) -> Self { + Self { stream, schema } + } +} + +impl Stream for FlightDataStream { + type Item = ArrowResult; + + fn poll_next( + mut self: std::pin::Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + self.stream.poll_next_unpin(cx).map(|x| match x { + Some(flight_data_chunk_result) => { + let converted_chunk = flight_data_chunk_result + .map_err(|e| ArrowError::from_external_error(Box::new(e))) + .and_then(|flight_data_chunk| { + flight_data_to_arrow_batch(&flight_data_chunk, self.schema.clone(), &[]) + }); + Some(converted_chunk) + } + None => None, + }) + } +} + +impl RecordBatchStream for FlightDataStream { + fn schema(&self) -> SchemaRef { + self.schema.clone() + } +} diff --git a/rust/ballista/rust/core/src/datasource.rs b/rust/ballista/rust/core/src/datasource.rs new file mode 100644 index 0000000000000..531f63df40e4b --- /dev/null +++ b/rust/ballista/rust/core/src/datasource.rs @@ -0,0 +1,71 @@ +// 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. + +use std::{any::Any, sync::Arc}; + +use arrow::datatypes::SchemaRef; +use datafusion::error::Result as DFResult; +use datafusion::{ + datasource::{datasource::Statistics, TableProvider}, + logical_plan::{Expr, LogicalPlan}, + physical_plan::ExecutionPlan, +}; + +/// This ugly adapter is needed because we use DataFusion's logical plan when building queries +/// and when we register tables with DataFusion's `ExecutionContext` we need to provide a +/// TableProvider which is effectively a wrapper around a physical plan. We need to be able to +/// register tables so that we can create logical plans from SQL statements that reference these +/// tables. +pub struct DFTableAdapter { + /// DataFusion logical plan + pub logical_plan: LogicalPlan, + /// DataFusion execution plan + plan: Arc, +} + +impl DFTableAdapter { + pub fn new(logical_plan: LogicalPlan, plan: Arc) -> Self { + Self { logical_plan, plan } + } +} + +impl TableProvider for DFTableAdapter { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.plan.schema() + } + + fn scan( + &self, + _projection: &Option>, + _batch_size: usize, + _filters: &[Expr], + ) -> DFResult> { + Ok(self.plan.clone()) + } + + fn statistics(&self) -> Statistics { + Statistics { + num_rows: None, + total_byte_size: None, + column_statistics: None, + } + } +} diff --git a/rust/ballista/rust/core/src/error.rs b/rust/ballista/rust/core/src/error.rs new file mode 100644 index 0000000000000..6d629b274f8ca --- /dev/null +++ b/rust/ballista/rust/core/src/error.rs @@ -0,0 +1,166 @@ +// 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 error types + +use std::{ + error::Error, + fmt::{Display, Formatter}, + io, result, +}; + +use arrow::error::ArrowError; +use datafusion::error::DataFusionError; +use sqlparser::parser; + +pub type Result = result::Result; + +/// Ballista error +#[derive(Debug)] +pub enum BallistaError { + NotImplemented(String), + General(String), + Internal(String), + ArrowError(ArrowError), + DataFusionError(DataFusionError), + SqlError(parser::ParserError), + IoError(io::Error), + // ReqwestError(reqwest::Error), + //HttpError(http::Error), + // KubeAPIError(kube::error::Error), + // KubeAPIRequestError(k8s_openapi::RequestError), + // KubeAPIResponseError(k8s_openapi::ResponseError), + TonicError(tonic::transport::Error), + GrpcError(tonic::Status), + TokioError(tokio::task::JoinError), +} + +impl Into> for BallistaError { + fn into(self) -> Result { + Err(self) + } +} + +pub fn ballista_error(message: &str) -> BallistaError { + BallistaError::General(message.to_owned()) +} + +impl From for BallistaError { + fn from(e: String) -> Self { + BallistaError::General(e) + } +} + +impl From for BallistaError { + fn from(e: ArrowError) -> Self { + BallistaError::ArrowError(e) + } +} + +impl From for BallistaError { + fn from(e: parser::ParserError) -> Self { + BallistaError::SqlError(e) + } +} + +impl From for BallistaError { + fn from(e: DataFusionError) -> Self { + BallistaError::DataFusionError(e) + } +} + +impl From for BallistaError { + fn from(e: io::Error) -> Self { + BallistaError::IoError(e) + } +} + +// impl From for BallistaError { +// fn from(e: reqwest::Error) -> Self { +// BallistaError::ReqwestError(e) +// } +// } +// +// impl From for BallistaError { +// fn from(e: http::Error) -> Self { +// BallistaError::HttpError(e) +// } +// } + +// impl From for BallistaError { +// fn from(e: kube::error::Error) -> Self { +// BallistaError::KubeAPIError(e) +// } +// } + +// impl From for BallistaError { +// fn from(e: k8s_openapi::RequestError) -> Self { +// BallistaError::KubeAPIRequestError(e) +// } +// } + +// impl From for BallistaError { +// fn from(e: k8s_openapi::ResponseError) -> Self { +// BallistaError::KubeAPIResponseError(e) +// } +// } + +impl From for BallistaError { + fn from(e: tonic::transport::Error) -> Self { + BallistaError::TonicError(e) + } +} + +impl From for BallistaError { + fn from(e: tonic::Status) -> Self { + BallistaError::GrpcError(e) + } +} + +impl From for BallistaError { + fn from(e: tokio::task::JoinError) -> Self { + BallistaError::TokioError(e) + } +} + +impl Display for BallistaError { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + match self { + BallistaError::NotImplemented(ref desc) => write!(f, "Not implemented: {}", desc), + BallistaError::General(ref desc) => write!(f, "General error: {}", desc), + BallistaError::ArrowError(ref desc) => write!(f, "Arrow error: {}", desc), + BallistaError::DataFusionError(ref desc) => write!(f, "DataFusion error: {:?}", desc), + BallistaError::SqlError(ref desc) => write!(f, "SQL error: {:?}", desc), + BallistaError::IoError(ref desc) => write!(f, "IO error: {}", desc), + // BallistaError::ReqwestError(ref desc) => write!(f, "Reqwest error: {}", desc), + // BallistaError::HttpError(ref desc) => write!(f, "HTTP error: {}", desc), + // BallistaError::KubeAPIError(ref desc) => write!(f, "Kube API error: {}", desc), + // BallistaError::KubeAPIRequestError(ref desc) => { + // write!(f, "KubeAPI request error: {}", desc) + // } + // BallistaError::KubeAPIResponseError(ref desc) => { + // write!(f, "KubeAPI response error: {}", desc) + // } + BallistaError::TonicError(desc) => write!(f, "Tonic error: {}", desc), + BallistaError::GrpcError(desc) => write!(f, "Grpc error: {}", desc), + BallistaError::Internal(desc) => write!(f, "Internal Ballista error: {}", desc), + BallistaError::TokioError(desc) => write!(f, "Tokio join error: {}", desc), + } + } +} + +impl Error for BallistaError {} diff --git a/rust/ballista/rust/core/src/execution_plans/mod.rs b/rust/ballista/rust/core/src/execution_plans/mod.rs new file mode 100644 index 0000000000000..1fb2010bd5456 --- /dev/null +++ b/rust/ballista/rust/core/src/execution_plans/mod.rs @@ -0,0 +1,27 @@ +// 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. + +//! This module contains execution plans that are needed to distribute Datafusion's execution plans into +//! several Ballista executors. + +mod query_stage; +mod shuffle_reader; +mod unresolved_shuffle; + +pub use query_stage::QueryStageExec; +pub use shuffle_reader::ShuffleReaderExec; +pub use unresolved_shuffle::UnresolvedShuffleExec; diff --git a/rust/ballista/rust/core/src/execution_plans/query_stage.rs b/rust/ballista/rust/core/src/execution_plans/query_stage.rs new file mode 100644 index 0000000000000..bb5878b81822f --- /dev/null +++ b/rust/ballista/rust/core/src/execution_plans/query_stage.rs @@ -0,0 +1,88 @@ +// 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. + +use std::sync::Arc; +use std::{any::Any, pin::Pin}; + +use arrow::datatypes::SchemaRef; +use async_trait::async_trait; +use datafusion::physical_plan::{ExecutionPlan, Partitioning}; +use datafusion::{error::Result, physical_plan::RecordBatchStream}; +use uuid::Uuid; + +/// QueryStageExec represents a section of a query plan that has consistent partitioning and +/// can be executed as one unit with each partition being executed in parallel. The output of +/// a query stage either forms the input of another query stage or can be the final result of +/// a query. +#[derive(Debug, Clone)] +pub struct QueryStageExec { + /// Unique ID for the job (query) that this stage is a part of + pub job_id: String, + /// Unique query stage ID within the job + pub stage_id: usize, + /// Physical execution plan for this query stage + pub child: Arc, +} + +impl QueryStageExec { + /// Create a new query stage + pub fn try_new(job_id: String, stage_id: usize, child: Arc) -> Result { + Ok(Self { + job_id, + stage_id, + child, + }) + } +} + +#[async_trait] +impl ExecutionPlan for QueryStageExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.child.schema() + } + + fn output_partitioning(&self) -> Partitioning { + self.child.output_partitioning() + } + + fn children(&self) -> Vec> { + vec![self.child.clone()] + } + + fn with_new_children( + &self, + children: Vec>, + ) -> Result> { + assert!(children.len() == 1); + Ok(Arc::new(QueryStageExec::try_new( + self.job_id.clone(), + self.stage_id, + children[0].clone(), + )?)) + } + + async fn execute( + &self, + partition: usize, + ) -> Result>> { + self.child.execute(partition).await + } +} diff --git a/rust/ballista/rust/core/src/execution_plans/shuffle_reader.rs b/rust/ballista/rust/core/src/execution_plans/shuffle_reader.rs new file mode 100644 index 0000000000000..e4736138601e8 --- /dev/null +++ b/rust/ballista/rust/core/src/execution_plans/shuffle_reader.rs @@ -0,0 +1,103 @@ +// 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. + +use std::sync::Arc; +use std::{any::Any, pin::Pin}; + +use crate::client::BallistaClient; +use crate::memory_stream::MemoryStream; +use crate::serde::scheduler::PartitionLocation; + +use arrow::datatypes::SchemaRef; +use async_trait::async_trait; +use datafusion::physical_plan::{ExecutionPlan, Partitioning}; +use datafusion::{ + error::{DataFusionError, Result}, + physical_plan::RecordBatchStream, +}; +use log::info; + +/// ShuffleReaderExec reads partitions that have already been materialized by an executor. +#[derive(Debug, Clone)] +pub struct ShuffleReaderExec { + // The query stage that is responsible for producing the shuffle partitions that + // this operator will read + pub(crate) partition_location: Vec, + pub(crate) schema: SchemaRef, +} + +impl ShuffleReaderExec { + /// Create a new ShuffleReaderExec + pub fn try_new(partition_meta: Vec, schema: SchemaRef) -> Result { + Ok(Self { + partition_location: partition_meta, + schema, + }) + } +} + +#[async_trait] +impl ExecutionPlan for ShuffleReaderExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + fn output_partitioning(&self) -> Partitioning { + Partitioning::UnknownPartitioning(self.partition_location.len()) + } + + fn children(&self) -> Vec> { + vec![] + } + + fn with_new_children( + &self, + _children: Vec>, + ) -> Result> { + Err(DataFusionError::Plan( + "Ballista ShuffleReaderExec does not support with_new_children()".to_owned(), + )) + } + + async fn execute( + &self, + partition: usize, + ) -> Result>> { + info!("ShuffleReaderExec::execute({})", partition); + let partition_location = &self.partition_location[partition]; + + let mut client = BallistaClient::try_new( + &partition_location.executor_meta.host, + partition_location.executor_meta.port, + ) + .await + .map_err(|e| DataFusionError::Execution(format!("Ballista Error: {:?}", e)))?; + + client + .fetch_partition( + &partition_location.partition_id.job_id, + partition_location.partition_id.stage_id, + partition, + ) + .await + .map_err(|e| DataFusionError::Execution(format!("Ballista Error: {:?}", e))) + } +} diff --git a/rust/ballista/rust/core/src/execution_plans/unresolved_shuffle.rs b/rust/ballista/rust/core/src/execution_plans/unresolved_shuffle.rs new file mode 100644 index 0000000000000..7fe39375aefbd --- /dev/null +++ b/rust/ballista/rust/core/src/execution_plans/unresolved_shuffle.rs @@ -0,0 +1,96 @@ +// 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. + +use std::sync::Arc; +use std::{any::Any, pin::Pin}; + +use crate::client::BallistaClient; +use crate::memory_stream::MemoryStream; +use crate::serde::scheduler::PartitionLocation; + +use arrow::datatypes::SchemaRef; +use async_trait::async_trait; +use datafusion::physical_plan::{ExecutionPlan, Partitioning}; +use datafusion::{ + error::{DataFusionError, Result}, + physical_plan::RecordBatchStream, +}; +use log::info; + +/// UnresolvedShuffleExec represents a dependency on the results of several QueryStageExec nodes which haven't been computed yet. +/// +/// An ExecutionPlan that contains an UnresolvedShuffleExec isn't ready for execution. The presence of this ExecutionPlan +/// is used as a signal so the scheduler knows it can't start computation on a specific QueryStageExec. +#[derive(Debug, Clone)] +pub struct UnresolvedShuffleExec { + // The query stage ids which needs to be computed + pub query_stage_ids: Vec, + + // The schema this node will have once it is replaced with a ShuffleReaderExec + pub schema: SchemaRef, + + // The partition count this node will have once it is replaced with a ShuffleReaderExec + pub partition_count: usize, +} + +impl UnresolvedShuffleExec { + /// Create a new UnresolvedShuffleExec + pub fn new(query_stage_ids: Vec, schema: SchemaRef, partition_count: usize) -> Self { + Self { + query_stage_ids, + schema, + partition_count, + } + } +} + +#[async_trait] +impl ExecutionPlan for UnresolvedShuffleExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + fn output_partitioning(&self) -> Partitioning { + Partitioning::UnknownPartitioning(self.partition_count) + } + + fn children(&self) -> Vec> { + vec![] + } + + fn with_new_children( + &self, + _children: Vec>, + ) -> Result> { + Err(DataFusionError::Plan( + "Ballista UnresolvedShuffleExec does not support with_new_children()".to_owned(), + )) + } + + async fn execute( + &self, + _partition: usize, + ) -> Result>> { + Err(DataFusionError::Plan( + "Ballista UnresolvedShuffleExec does not support execution".to_owned(), + )) + } +} diff --git a/rust/ballista/rust/core/src/lib.rs b/rust/ballista/rust/core/src/lib.rs new file mode 100644 index 0000000000000..425dbab34c132 --- /dev/null +++ b/rust/ballista/rust/core/src/lib.rs @@ -0,0 +1,34 @@ +// 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 Distributed Compute +#![allow(unused_imports)] +pub const BALLISTA_VERSION: &str = env!("CARGO_PKG_VERSION"); + +pub fn print_version() { + println!("Ballista version: {}", BALLISTA_VERSION) +} + +pub mod client; +pub mod datasource; +pub mod error; +pub mod execution_plans; +pub mod memory_stream; +pub mod utils; + +#[macro_use] +pub mod serde; diff --git a/rust/ballista/rust/core/src/memory_stream.rs b/rust/ballista/rust/core/src/memory_stream.rs new file mode 100644 index 0000000000000..8bf5e203f6d14 --- /dev/null +++ b/rust/ballista/rust/core/src/memory_stream.rs @@ -0,0 +1,93 @@ +// 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. + +//! This is copied from DataFusion because it is declared as `pub(crate)`. See +//! https://issues.apache.org/jira/browse/ARROW-11276. + +use std::task::{Context, Poll}; + +use arrow::{datatypes::SchemaRef, error::Result, record_batch::RecordBatch}; +use datafusion::physical_plan::RecordBatchStream; +use futures::Stream; + +/// Iterator over batches + +pub struct MemoryStream { + /// Vector of record batches + data: Vec, + /// Schema representing the data + schema: SchemaRef, + /// Optional projection for which columns to load + projection: Option>, + /// Index into the data + index: usize, +} + +impl MemoryStream { + /// Create an iterator for a vector of record batches + + pub fn try_new( + data: Vec, + schema: SchemaRef, + projection: Option>, + ) -> Result { + Ok(Self { + data, + schema, + projection, + index: 0, + }) + } +} + +impl Stream for MemoryStream { + type Item = Result; + + fn poll_next( + mut self: std::pin::Pin<&mut Self>, + _: &mut Context<'_>, + ) -> Poll> { + Poll::Ready(if self.index < self.data.len() { + self.index += 1; + + let batch = &self.data[self.index - 1]; + + // apply projection + match &self.projection { + Some(columns) => Some(RecordBatch::try_new( + self.schema.clone(), + columns.iter().map(|i| batch.column(*i).clone()).collect(), + )), + None => Some(Ok(batch.clone())), + } + } else { + None + }) + } + + fn size_hint(&self) -> (usize, Option) { + (self.data.len(), Some(self.data.len())) + } +} + +impl RecordBatchStream for MemoryStream { + /// Get the schema + + fn schema(&self) -> SchemaRef { + self.schema.clone() + } +} diff --git a/rust/ballista/rust/core/src/serde/logical_plan/from_proto.rs b/rust/ballista/rust/core/src/serde/logical_plan/from_proto.rs new file mode 100644 index 0000000000000..53ffef4bf708e --- /dev/null +++ b/rust/ballista/rust/core/src/serde/logical_plan/from_proto.rs @@ -0,0 +1,1060 @@ +// 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. + +//! Serde code to convert from protocol buffers to Rust data structures. + +use std::{ + convert::{From, TryInto}, + unimplemented, +}; + +use crate::error::BallistaError; +use crate::serde::{proto_error, protobuf}; +use crate::{convert_box_required, convert_required}; + +use arrow::datatypes::{DataType, Field, Schema}; +use datafusion::logical_plan::{ + abs, acos, asin, atan, ceil, cos, exp, floor, log10, log2, round, signum, sin, sqrt, tan, + trunc, Expr, JoinType, LogicalPlan, LogicalPlanBuilder, Operator, +}; +use datafusion::physical_plan::aggregates::AggregateFunction; +use datafusion::physical_plan::csv::CsvReadOptions; +use datafusion::scalar::ScalarValue; +use protobuf::logical_plan_node::LogicalPlanType; +use protobuf::{logical_expr_node::ExprType, scalar_type}; + +// use uuid::Uuid; + +impl TryInto for &protobuf::LogicalPlanNode { + type Error = BallistaError; + + fn try_into(self) -> Result { + let plan = self.logical_plan_type.as_ref().ok_or_else(|| { + proto_error(format!( + "logical_plan::from_proto() Unsupported logical plan '{:?}'", + self + )) + })?; + match plan { + LogicalPlanType::Projection(projection) => { + let input: LogicalPlan = convert_box_required!(projection.input)?; + LogicalPlanBuilder::from(&input) + .project( + &projection + .expr + .iter() + .map(|expr| expr.try_into()) + .collect::, _>>()?, + )? + .build() + .map_err(|e| e.into()) + } + LogicalPlanType::Selection(selection) => { + let input: LogicalPlan = convert_box_required!(selection.input)?; + LogicalPlanBuilder::from(&input) + .filter( + selection + .expr + .as_ref() + .expect("expression required") + .try_into()?, + )? + .build() + .map_err(|e| e.into()) + } + LogicalPlanType::Aggregate(aggregate) => { + let input: LogicalPlan = convert_box_required!(aggregate.input)?; + let group_expr = aggregate + .group_expr + .iter() + .map(|expr| expr.try_into()) + .collect::, _>>()?; + let aggr_expr = aggregate + .aggr_expr + .iter() + .map(|expr| expr.try_into()) + .collect::, _>>()?; + LogicalPlanBuilder::from(&input) + .aggregate(&group_expr, &aggr_expr)? + .build() + .map_err(|e| e.into()) + } + LogicalPlanType::CsvScan(scan) => { + let schema: Schema = convert_required!(scan.schema)?; + let options = CsvReadOptions::new() + .schema(&schema) + .delimiter(scan.delimiter.as_bytes()[0]) + .file_extension(&scan.file_extension) + .has_header(scan.has_header); + + let mut projection = None; + if let Some(column_names) = &scan.projection { + let column_indices = column_names + .columns + .iter() + .map(|name| schema.index_of(name)) + .collect::, _>>()?; + projection = Some(column_indices); + } + + LogicalPlanBuilder::scan_csv(&scan.path, options, projection)? + .build() + .map_err(|e| e.into()) + } + LogicalPlanType::ParquetScan(scan) => { + let projection = match scan.projection.as_ref() { + None => None, + Some(columns) => { + let schema: Schema = convert_required!(scan.schema)?; + let r: Result, _> = columns + .columns + .iter() + .map(|col_name| { + schema.fields().iter().position(|field| field.name() == col_name).ok_or_else(|| { + let column_names: Vec<&String> = schema.fields().iter().map(|f| f.name()).collect(); + proto_error(format!( + "Parquet projection contains column name that is not present in schema. Column name: {}. Schema columns: {:?}", + col_name, column_names + )) + }) + }) + .collect(); + Some(r?) + } + }; + LogicalPlanBuilder::scan_parquet(&scan.path, projection, 24)? //TODO concurrency + .build() + .map_err(|e| e.into()) + } + LogicalPlanType::Sort(sort) => { + let input: LogicalPlan = convert_box_required!(sort.input)?; + let sort_expr: Vec = sort + .expr + .iter() + .map(|expr| expr.try_into()) + .collect::, _>>()?; + LogicalPlanBuilder::from(&input) + .sort(&sort_expr)? + .build() + .map_err(|e| e.into()) + } + LogicalPlanType::Repartition(repartition) => { + use datafusion::logical_plan::Partitioning; + let input: LogicalPlan = convert_box_required!(repartition.input)?; + use protobuf::repartition_node::PartitionMethod; + let pb_partition_method = repartition.partition_method.clone().ok_or_else(|| { + BallistaError::General(String::from( + "Protobuf deserialization error, RepartitionNode was missing required field 'partition_method'", + )) + })?; + + let partitioning_scheme = match pb_partition_method { + PartitionMethod::Hash(protobuf::HashRepartition { + hash_expr: pb_hash_expr, + partition_count, + }) => Partitioning::Hash( + pb_hash_expr + .iter() + .map(|pb_expr| pb_expr.try_into()) + .collect::, _>>()?, + partition_count as usize, + ), + PartitionMethod::RoundRobin(batch_size) => { + Partitioning::RoundRobinBatch(batch_size as usize) + } + }; + + LogicalPlanBuilder::from(&input) + .repartition(partitioning_scheme)? + .build() + .map_err(|e| e.into()) + } + LogicalPlanType::EmptyRelation(empty_relation) => { + LogicalPlanBuilder::empty(empty_relation.produce_one_row) + .build() + .map_err(|e| e.into()) + } + LogicalPlanType::CreateExternalTable(create_extern_table) => { + let pb_schema = (create_extern_table.schema.clone()).ok_or_else(|| { + BallistaError::General(String::from( + "Protobuf deserialization error, CreateExternalTableNode was missing required field schema.", + )) + })?; + + let pb_file_type: protobuf::FileType = create_extern_table.file_type.try_into()?; + + Ok(LogicalPlan::CreateExternalTable { + schema: pb_schema.try_into()?, + name: create_extern_table.name.clone(), + location: create_extern_table.location.clone(), + file_type: pb_file_type.into(), + has_header: create_extern_table.has_header, + }) + } + LogicalPlanType::Explain(explain) => { + let input: LogicalPlan = convert_box_required!(explain.input)?; + LogicalPlanBuilder::from(&input) + .explain(explain.verbose)? + .build() + .map_err(|e| e.into()) + } + LogicalPlanType::Limit(limit) => { + let input: LogicalPlan = convert_box_required!(limit.input)?; + LogicalPlanBuilder::from(&input) + .limit(limit.limit as usize)? + .build() + .map_err(|e| e.into()) + } + LogicalPlanType::Join(join) => { + let left_keys: Vec<&str> = + join.left_join_column.iter().map(|i| i.as_str()).collect(); + let right_keys: Vec<&str> = + join.right_join_column.iter().map(|i| i.as_str()).collect(); + let join_type = protobuf::JoinType::from_i32(join.join_type).ok_or_else(|| { + proto_error(format!( + "Received a JoinNode message with unknown JoinType {}", + join.join_type + )) + })?; + let join_type = match join_type { + protobuf::JoinType::Inner => JoinType::Inner, + protobuf::JoinType::Left => JoinType::Left, + protobuf::JoinType::Right => JoinType::Right, + }; + LogicalPlanBuilder::from(&convert_box_required!(join.left)?) + .join( + &convert_box_required!(join.right)?, + join_type, + &left_keys, + &right_keys, + )? + .build() + .map_err(|e| e.into()) + } + } + } +} + +impl TryInto for protobuf::Schema { + type Error = BallistaError; + fn try_into(self) -> Result { + let schema: Schema = (&self).try_into()?; + schema.try_into().map_err(BallistaError::DataFusionError) + } +} + +impl TryInto for protobuf::Schema { + type Error = BallistaError; + fn try_into(self) -> Result { + use datafusion::logical_plan::ToDFSchema; + let schema: Schema = (&self).try_into()?; + schema + .to_dfschema_ref() + .map_err(BallistaError::DataFusionError) + } +} + +impl TryInto for &protobuf::scalar_type::Datatype { + type Error = BallistaError; + fn try_into(self) -> Result { + use protobuf::scalar_type::Datatype; + Ok(match self { + Datatype::Scalar(scalar_type) => { + let pb_scalar_enum = protobuf::PrimitiveScalarType::from_i32(*scalar_type).ok_or_else(|| { + proto_error(format!( + "Protobuf deserialization error, scalar_type::Datatype missing was provided invalid enum variant: {}", + *scalar_type + )) + })?; + pb_scalar_enum.into() + } + Datatype::List(protobuf::ScalarListType { + deepest_type, + field_names, + }) => { + if field_names.is_empty() { + return Err(proto_error( + "Protobuf deserialization error: found no field names in ScalarListType message which requires at least one", + )); + } + let pb_scalar_type = protobuf::PrimitiveScalarType::from_i32(*deepest_type) + .ok_or_else(|| { + proto_error(format!( + "Protobuf deserialization error: invalid i32 for scalar enum: {}", + *deepest_type + )) + })?; + //Because length is checked above it is safe to unwrap .last() + let mut scalar_type = arrow::datatypes::DataType::List(Box::new(Field::new( + field_names.last().unwrap().as_str(), + pb_scalar_type.into(), + true, + ))); + //Iterate over field names in reverse order except for the last item in the vector + for name in field_names.iter().rev().skip(1) { + let new_datatype = arrow::datatypes::DataType::List(Box::new(Field::new( + name.as_str(), + scalar_type, + true, + ))); + scalar_type = new_datatype; + } + scalar_type + } + }) + } +} + +impl TryInto for &protobuf::arrow_type::ArrowTypeEnum { + type Error = BallistaError; + fn try_into(self) -> Result { + use arrow::datatypes::DataType; + use protobuf::arrow_type; + Ok(match self { + arrow_type::ArrowTypeEnum::None(_) => DataType::Null, + arrow_type::ArrowTypeEnum::Bool(_) => DataType::Boolean, + arrow_type::ArrowTypeEnum::Uint8(_) => DataType::UInt8, + arrow_type::ArrowTypeEnum::Int8(_) => DataType::Int8, + arrow_type::ArrowTypeEnum::Uint16(_) => DataType::UInt16, + arrow_type::ArrowTypeEnum::Int16(_) => DataType::Int16, + arrow_type::ArrowTypeEnum::Uint32(_) => DataType::UInt32, + arrow_type::ArrowTypeEnum::Int32(_) => DataType::Int32, + arrow_type::ArrowTypeEnum::Uint64(_) => DataType::UInt64, + arrow_type::ArrowTypeEnum::Int64(_) => DataType::Int64, + arrow_type::ArrowTypeEnum::Float16(_) => DataType::Float16, + arrow_type::ArrowTypeEnum::Float32(_) => DataType::Float32, + arrow_type::ArrowTypeEnum::Float64(_) => DataType::Float64, + arrow_type::ArrowTypeEnum::Utf8(_) => DataType::Utf8, + arrow_type::ArrowTypeEnum::LargeUtf8(_) => DataType::LargeUtf8, + arrow_type::ArrowTypeEnum::Binary(_) => DataType::Binary, + arrow_type::ArrowTypeEnum::FixedSizeBinary(size) => DataType::FixedSizeBinary(*size), + arrow_type::ArrowTypeEnum::LargeBinary(_) => DataType::LargeBinary, + arrow_type::ArrowTypeEnum::Date32(_) => DataType::Date32, + arrow_type::ArrowTypeEnum::Date64(_) => DataType::Date64, + arrow_type::ArrowTypeEnum::Duration(time_unit) => { + DataType::Duration(protobuf::TimeUnit::from_i32_to_arrow(*time_unit)?) + } + arrow_type::ArrowTypeEnum::Timestamp(protobuf::Timestamp { + time_unit, + timezone, + }) => DataType::Timestamp( + protobuf::TimeUnit::from_i32_to_arrow(*time_unit)?, + match timezone.len() { + 0 => None, + _ => Some(timezone.to_owned()), + }, + ), + arrow_type::ArrowTypeEnum::Time32(time_unit) => { + DataType::Time32(protobuf::TimeUnit::from_i32_to_arrow(*time_unit)?) + } + arrow_type::ArrowTypeEnum::Time64(time_unit) => { + DataType::Time64(protobuf::TimeUnit::from_i32_to_arrow(*time_unit)?) + } + arrow_type::ArrowTypeEnum::Interval(interval_unit) => { + DataType::Interval(protobuf::IntervalUnit::from_i32_to_arrow(*interval_unit)?) + } + arrow_type::ArrowTypeEnum::Decimal(protobuf::Decimal { whole, fractional }) => { + DataType::Decimal(*whole as usize, *fractional as usize) + } + arrow_type::ArrowTypeEnum::List(list) => { + let list_type: &protobuf::Field = list + .as_ref() + .field_type + .as_ref() + .ok_or_else(|| proto_error("Protobuf deserialization error: List message missing required field 'field_type'"))? + .as_ref(); + DataType::List(Box::new(list_type.try_into()?)) + } + arrow_type::ArrowTypeEnum::LargeList(list) => { + let list_type: &protobuf::Field = list + .as_ref() + .field_type + .as_ref() + .ok_or_else(|| proto_error("Protobuf deserialization error: List message missing required field 'field_type'"))? + .as_ref(); + DataType::LargeList(Box::new(list_type.try_into()?)) + } + arrow_type::ArrowTypeEnum::FixedSizeList(list) => { + let list_type: &protobuf::Field = list + .as_ref() + .field_type + .as_ref() + .ok_or_else(|| proto_error("Protobuf deserialization error: List message missing required field 'field_type'"))? + .as_ref(); + let list_size = list.list_size; + DataType::FixedSizeList(Box::new(list_type.try_into()?), list_size) + } + arrow_type::ArrowTypeEnum::Struct(strct) => DataType::Struct( + strct + .sub_field_types + .iter() + .map(|field| field.try_into()) + .collect::, _>>()?, + ), + arrow_type::ArrowTypeEnum::Union(union) => DataType::Union( + union + .union_types + .iter() + .map(|field| field.try_into()) + .collect::, _>>()?, + ), + arrow_type::ArrowTypeEnum::Dictionary(dict) => { + let pb_key_datatype = dict + .as_ref() + .key + .as_ref() + .ok_or_else(|| proto_error("Protobuf deserialization error: Dictionary message missing required field 'key'"))?; + let pb_value_datatype = dict + .as_ref() + .value + .as_ref() + .ok_or_else(|| proto_error("Protobuf deserialization error: Dictionary message missing required field 'key'"))?; + let key_datatype: DataType = pb_key_datatype.as_ref().try_into()?; + let value_datatype: DataType = pb_value_datatype.as_ref().try_into()?; + DataType::Dictionary(Box::new(key_datatype), Box::new(value_datatype)) + } + }) + } +} + +impl Into for protobuf::PrimitiveScalarType { + fn into(self) -> arrow::datatypes::DataType { + use arrow::datatypes::DataType; + match self { + protobuf::PrimitiveScalarType::Bool => DataType::Boolean, + protobuf::PrimitiveScalarType::Uint8 => DataType::UInt8, + protobuf::PrimitiveScalarType::Int8 => DataType::Int8, + protobuf::PrimitiveScalarType::Uint16 => DataType::UInt16, + protobuf::PrimitiveScalarType::Int16 => DataType::Int16, + protobuf::PrimitiveScalarType::Uint32 => DataType::UInt32, + protobuf::PrimitiveScalarType::Int32 => DataType::Int32, + protobuf::PrimitiveScalarType::Uint64 => DataType::UInt64, + protobuf::PrimitiveScalarType::Int64 => DataType::Int64, + protobuf::PrimitiveScalarType::Float32 => DataType::Float32, + protobuf::PrimitiveScalarType::Float64 => DataType::Float64, + protobuf::PrimitiveScalarType::Utf8 => DataType::Utf8, + protobuf::PrimitiveScalarType::LargeUtf8 => DataType::LargeUtf8, + protobuf::PrimitiveScalarType::Date32 => DataType::Date32, + protobuf::PrimitiveScalarType::TimeMicrosecond => { + DataType::Time64(arrow::datatypes::TimeUnit::Microsecond) + } + protobuf::PrimitiveScalarType::TimeNanosecond => { + DataType::Time64(arrow::datatypes::TimeUnit::Nanosecond) + } + protobuf::PrimitiveScalarType::Null => DataType::Null, + } + } +} + +//Does not typecheck lists +fn typechecked_scalar_value_conversion( + tested_type: &protobuf::scalar_value::Value, + required_type: protobuf::PrimitiveScalarType, +) -> Result { + use protobuf::scalar_value::Value; + use protobuf::PrimitiveScalarType; + Ok(match (tested_type, &required_type) { + (Value::BoolValue(v), PrimitiveScalarType::Bool) => ScalarValue::Boolean(Some(*v)), + (Value::Int8Value(v), PrimitiveScalarType::Int8) => ScalarValue::Int8(Some(*v as i8)), + (Value::Int16Value(v), PrimitiveScalarType::Int16) => ScalarValue::Int16(Some(*v as i16)), + (Value::Int32Value(v), PrimitiveScalarType::Int32) => ScalarValue::Int32(Some(*v)), + (Value::Int64Value(v), PrimitiveScalarType::Int64) => ScalarValue::Int64(Some(*v)), + (Value::Uint8Value(v), PrimitiveScalarType::Uint8) => ScalarValue::UInt8(Some(*v as u8)), + (Value::Uint16Value(v), PrimitiveScalarType::Uint16) => { + ScalarValue::UInt16(Some(*v as u16)) + } + (Value::Uint32Value(v), PrimitiveScalarType::Uint32) => ScalarValue::UInt32(Some(*v)), + (Value::Uint64Value(v), PrimitiveScalarType::Uint64) => ScalarValue::UInt64(Some(*v)), + (Value::Float32Value(v), PrimitiveScalarType::Float32) => ScalarValue::Float32(Some(*v)), + (Value::Float64Value(v), PrimitiveScalarType::Float64) => ScalarValue::Float64(Some(*v)), + (Value::Date32Value(v), PrimitiveScalarType::Date32) => ScalarValue::Date32(Some(*v)), + (Value::TimeMicrosecondValue(v), PrimitiveScalarType::TimeMicrosecond) => { + ScalarValue::TimeMicrosecond(Some(*v)) + } + (Value::TimeNanosecondValue(v), PrimitiveScalarType::TimeMicrosecond) => { + ScalarValue::TimeNanosecond(Some(*v)) + } + (Value::Utf8Value(v), PrimitiveScalarType::Utf8) => ScalarValue::Utf8(Some(v.to_owned())), + (Value::LargeUtf8Value(v), PrimitiveScalarType::LargeUtf8) => { + ScalarValue::LargeUtf8(Some(v.to_owned())) + } + + (Value::NullValue(i32_enum), required_scalar_type) => { + if *i32_enum == *required_scalar_type as i32 { + let pb_scalar_type = PrimitiveScalarType::from_i32(*i32_enum).ok_or_else(|| { + BallistaError::General(format!( + "Invalid i32_enum={} when converting with PrimitiveScalarType::from_i32()", + *i32_enum + )) + })?; + let scalar_value: ScalarValue = match pb_scalar_type { + PrimitiveScalarType::Bool => ScalarValue::Boolean(None), + PrimitiveScalarType::Uint8 => ScalarValue::UInt8(None), + PrimitiveScalarType::Int8 => ScalarValue::Int8(None), + PrimitiveScalarType::Uint16 => ScalarValue::UInt16(None), + PrimitiveScalarType::Int16 => ScalarValue::Int16(None), + PrimitiveScalarType::Uint32 => ScalarValue::UInt32(None), + PrimitiveScalarType::Int32 => ScalarValue::Int32(None), + PrimitiveScalarType::Uint64 => ScalarValue::UInt64(None), + PrimitiveScalarType::Int64 => ScalarValue::Int64(None), + PrimitiveScalarType::Float32 => ScalarValue::Float32(None), + PrimitiveScalarType::Float64 => ScalarValue::Float64(None), + PrimitiveScalarType::Utf8 => ScalarValue::Utf8(None), + PrimitiveScalarType::LargeUtf8 => ScalarValue::LargeUtf8(None), + PrimitiveScalarType::Date32 => ScalarValue::Date32(None), + PrimitiveScalarType::TimeMicrosecond => ScalarValue::TimeMicrosecond(None), + PrimitiveScalarType::TimeNanosecond => ScalarValue::TimeNanosecond(None), + PrimitiveScalarType::Null => { + return Err(proto_error( + "Untyped scalar null is not a valid scalar value", + )) + } + }; + scalar_value + } else { + return Err(proto_error("Could not convert to the proper type")); + } + } + _ => return Err(proto_error("Could not convert to the proper type")), + }) +} + +impl TryInto for &protobuf::scalar_value::Value { + type Error = BallistaError; + fn try_into(self) -> Result { + use datafusion::scalar::ScalarValue; + use protobuf::PrimitiveScalarType; + let scalar = match self { + protobuf::scalar_value::Value::BoolValue(v) => ScalarValue::Boolean(Some(*v)), + protobuf::scalar_value::Value::Utf8Value(v) => ScalarValue::Utf8(Some(v.to_owned())), + protobuf::scalar_value::Value::LargeUtf8Value(v) => { + ScalarValue::LargeUtf8(Some(v.to_owned())) + } + protobuf::scalar_value::Value::Int8Value(v) => ScalarValue::Int8(Some(*v as i8)), + protobuf::scalar_value::Value::Int16Value(v) => ScalarValue::Int16(Some(*v as i16)), + protobuf::scalar_value::Value::Int32Value(v) => ScalarValue::Int32(Some(*v)), + protobuf::scalar_value::Value::Int64Value(v) => ScalarValue::Int64(Some(*v)), + protobuf::scalar_value::Value::Uint8Value(v) => ScalarValue::UInt8(Some(*v as u8)), + protobuf::scalar_value::Value::Uint16Value(v) => ScalarValue::UInt16(Some(*v as u16)), + protobuf::scalar_value::Value::Uint32Value(v) => ScalarValue::UInt32(Some(*v)), + protobuf::scalar_value::Value::Uint64Value(v) => ScalarValue::UInt64(Some(*v)), + protobuf::scalar_value::Value::Float32Value(v) => ScalarValue::Float32(Some(*v)), + protobuf::scalar_value::Value::Float64Value(v) => ScalarValue::Float64(Some(*v)), + protobuf::scalar_value::Value::Date32Value(v) => ScalarValue::Date32(Some(*v)), + protobuf::scalar_value::Value::TimeMicrosecondValue(v) => { + ScalarValue::TimeMicrosecond(Some(*v)) + } + protobuf::scalar_value::Value::TimeNanosecondValue(v) => { + ScalarValue::TimeNanosecond(Some(*v)) + } + protobuf::scalar_value::Value::ListValue(v) => v.try_into()?, + protobuf::scalar_value::Value::NullListValue(v) => { + ScalarValue::List(None, v.try_into()?) + } + protobuf::scalar_value::Value::NullValue(null_enum) => { + PrimitiveScalarType::from_i32(*null_enum) + .ok_or_else(|| proto_error("Invalid scalar type"))? + .try_into()? + } + }; + Ok(scalar) + } +} + +impl TryInto for &protobuf::ScalarListValue { + type Error = BallistaError; + fn try_into(self) -> Result { + use protobuf::scalar_type::Datatype; + use protobuf::PrimitiveScalarType; + let protobuf::ScalarListValue { datatype, values } = self; + let pb_scalar_type = datatype + .as_ref() + .ok_or_else(|| proto_error("Protobuf deserialization error: ScalarListValue messsage missing required field 'datatype'"))?; + let scalar_type = pb_scalar_type + .datatype + .as_ref() + .ok_or_else(|| proto_error("Protobuf deserialization error: ScalarListValue.Datatype messsage missing required field 'datatype'"))?; + let scalar_values = match scalar_type { + Datatype::Scalar(scalar_type_i32) => { + let leaf_scalar_type = protobuf::PrimitiveScalarType::from_i32(*scalar_type_i32) + .ok_or_else(|| proto_error("Error converting i32 to basic scalar type"))?; + let typechecked_values: Vec = values + .iter() + .map(|protobuf::ScalarValue { value: opt_value }| { + let value = opt_value.as_ref().ok_or_else(|| { + proto_error( + "Protobuf deserialization error: missing required field 'value'", + ) + })?; + typechecked_scalar_value_conversion(value, leaf_scalar_type) + }) + .collect::, _>>()?; + datafusion::scalar::ScalarValue::List( + Some(typechecked_values), + leaf_scalar_type.into(), + ) + } + Datatype::List(list_type) => { + let protobuf::ScalarListType { + deepest_type, + field_names, + } = &list_type; + let leaf_type = PrimitiveScalarType::from_i32(*deepest_type) + .ok_or_else(|| proto_error("Error converting i32 to basic scalar type"))?; + let depth = field_names.len(); + + let typechecked_values: Vec = if depth == 0 { + return Err(proto_error( + "Protobuf deserialization error, ScalarListType had no field names, requires at least one", + )); + } else if depth == 1 { + values + .iter() + .map(|protobuf::ScalarValue { value: opt_value }| { + let value = opt_value + .as_ref() + .ok_or_else(|| proto_error("Protobuf deserialization error: missing required field 'value'"))?; + typechecked_scalar_value_conversion(value, leaf_type) + }) + .collect::, _>>()? + } else { + values + .iter() + .map(|protobuf::ScalarValue { value: opt_value }| { + let value = opt_value + .as_ref() + .ok_or_else(|| proto_error("Protobuf deserialization error: missing required field 'value'"))?; + value.try_into() + }) + .collect::, _>>()? + }; + datafusion::scalar::ScalarValue::List( + match typechecked_values.len() { + 0 => None, + _ => Some(typechecked_values), + }, + list_type.try_into()?, + ) + } + }; + Ok(scalar_values) + } +} + +impl TryInto for &protobuf::ScalarListType { + type Error = BallistaError; + fn try_into(self) -> Result { + use protobuf::PrimitiveScalarType; + let protobuf::ScalarListType { + deepest_type, + field_names, + } = self; + + let depth = field_names.len(); + if depth == 0 { + return Err(proto_error( + "Protobuf deserialization error: Found a ScalarListType message with no field names, at least one is required", + )); + } + + let mut curr_type = arrow::datatypes::DataType::List(Box::new(Field::new( + //Since checked vector is not empty above this is safe to unwrap + field_names.last().unwrap(), + PrimitiveScalarType::from_i32(*deepest_type) + .ok_or_else(|| proto_error("Could not convert to datafusion scalar type"))? + .into(), + true, + ))); + //Iterates over field names in reverse order except for the last item in the vector + for name in field_names.iter().rev().skip(1) { + let temp_curr_type = + arrow::datatypes::DataType::List(Box::new(Field::new(name, curr_type, true))); + curr_type = temp_curr_type; + } + Ok(curr_type) + } +} + +impl TryInto for protobuf::PrimitiveScalarType { + type Error = BallistaError; + fn try_into(self) -> Result { + use datafusion::scalar::ScalarValue; + Ok(match self { + protobuf::PrimitiveScalarType::Null => { + return Err(proto_error("Untyped null is an invalid scalar value")) + } + protobuf::PrimitiveScalarType::Bool => ScalarValue::Boolean(None), + protobuf::PrimitiveScalarType::Uint8 => ScalarValue::UInt8(None), + protobuf::PrimitiveScalarType::Int8 => ScalarValue::Int8(None), + protobuf::PrimitiveScalarType::Uint16 => ScalarValue::UInt16(None), + protobuf::PrimitiveScalarType::Int16 => ScalarValue::Int16(None), + protobuf::PrimitiveScalarType::Uint32 => ScalarValue::UInt32(None), + protobuf::PrimitiveScalarType::Int32 => ScalarValue::Int32(None), + protobuf::PrimitiveScalarType::Uint64 => ScalarValue::UInt64(None), + protobuf::PrimitiveScalarType::Int64 => ScalarValue::Int64(None), + protobuf::PrimitiveScalarType::Float32 => ScalarValue::Float32(None), + protobuf::PrimitiveScalarType::Float64 => ScalarValue::Float64(None), + protobuf::PrimitiveScalarType::Utf8 => ScalarValue::Utf8(None), + protobuf::PrimitiveScalarType::LargeUtf8 => ScalarValue::LargeUtf8(None), + protobuf::PrimitiveScalarType::Date32 => ScalarValue::Date32(None), + protobuf::PrimitiveScalarType::TimeMicrosecond => ScalarValue::TimeMicrosecond(None), + protobuf::PrimitiveScalarType::TimeNanosecond => ScalarValue::TimeNanosecond(None), + }) + } +} + +impl TryInto for &protobuf::ScalarValue { + type Error = BallistaError; + fn try_into(self) -> Result { + let value = self.value.as_ref().ok_or_else(|| { + proto_error("Protobuf deserialization error: missing required field 'value'") + })?; + Ok(match value { + protobuf::scalar_value::Value::BoolValue(v) => ScalarValue::Boolean(Some(*v)), + protobuf::scalar_value::Value::Utf8Value(v) => ScalarValue::Utf8(Some(v.to_owned())), + protobuf::scalar_value::Value::LargeUtf8Value(v) => { + ScalarValue::LargeUtf8(Some(v.to_owned())) + } + protobuf::scalar_value::Value::Int8Value(v) => ScalarValue::Int8(Some(*v as i8)), + protobuf::scalar_value::Value::Int16Value(v) => ScalarValue::Int16(Some(*v as i16)), + protobuf::scalar_value::Value::Int32Value(v) => ScalarValue::Int32(Some(*v)), + protobuf::scalar_value::Value::Int64Value(v) => ScalarValue::Int64(Some(*v)), + protobuf::scalar_value::Value::Uint8Value(v) => ScalarValue::UInt8(Some(*v as u8)), + protobuf::scalar_value::Value::Uint16Value(v) => ScalarValue::UInt16(Some(*v as u16)), + protobuf::scalar_value::Value::Uint32Value(v) => ScalarValue::UInt32(Some(*v)), + protobuf::scalar_value::Value::Uint64Value(v) => ScalarValue::UInt64(Some(*v)), + protobuf::scalar_value::Value::Float32Value(v) => ScalarValue::Float32(Some(*v)), + protobuf::scalar_value::Value::Float64Value(v) => ScalarValue::Float64(Some(*v)), + protobuf::scalar_value::Value::Date32Value(v) => ScalarValue::Date32(Some(*v)), + protobuf::scalar_value::Value::TimeMicrosecondValue(v) => { + ScalarValue::TimeMicrosecond(Some(*v)) + } + protobuf::scalar_value::Value::TimeNanosecondValue(v) => { + ScalarValue::TimeNanosecond(Some(*v)) + } + protobuf::scalar_value::Value::ListValue(scalar_list) => { + let protobuf::ScalarListValue { + values, + datatype: opt_scalar_type, + } = &scalar_list; + let pb_scalar_type = opt_scalar_type + .as_ref() + .ok_or_else(|| proto_error("Protobuf deserialization err: ScalaListValue missing required field 'datatype'"))?; + let typechecked_values: Vec = values + .iter() + .map(|val| val.try_into()) + .collect::, _>>()?; + let scalar_type: arrow::datatypes::DataType = pb_scalar_type.try_into()?; + ScalarValue::List(Some(typechecked_values), scalar_type) + } + protobuf::scalar_value::Value::NullListValue(v) => { + let pb_datatype = v + .datatype + .as_ref() + .ok_or_else(|| proto_error("Protobuf deserialization error: NullListValue message missing required field 'datatyp'"))?; + ScalarValue::List(None, pb_datatype.try_into()?) + } + protobuf::scalar_value::Value::NullValue(v) => { + let null_type_enum = protobuf::PrimitiveScalarType::from_i32(*v) + .ok_or_else(|| proto_error("Protobuf deserialization error found invalid enum variant for DatafusionScalar"))?; + null_type_enum.try_into()? + } + }) + } +} + +impl TryInto for &protobuf::LogicalExprNode { + type Error = BallistaError; + + fn try_into(self) -> Result { + use protobuf::logical_expr_node::ExprType; + + let expr_type = self + .expr_type + .as_ref() + .ok_or_else(|| proto_error("Unexpected empty logical expression"))?; + match expr_type { + ExprType::BinaryExpr(binary_expr) => Ok(Expr::BinaryExpr { + left: Box::new(parse_required_expr(&binary_expr.l)?), + op: from_proto_binary_op(&binary_expr.op)?, + right: Box::new(parse_required_expr(&binary_expr.r)?), + }), + ExprType::ColumnName(column_name) => Ok(Expr::Column(column_name.to_owned())), + ExprType::Literal(literal) => { + use datafusion::scalar::ScalarValue; + let scalar_value: datafusion::scalar::ScalarValue = literal.try_into()?; + Ok(Expr::Literal(scalar_value)) + } + ExprType::AggregateExpr(expr) => { + let aggr_function = protobuf::AggregateFunction::from_i32(expr.aggr_function) + .ok_or_else(|| { + proto_error(format!( + "Received an unknown aggregate function: {}", + expr.aggr_function + )) + })?; + let fun = match aggr_function { + protobuf::AggregateFunction::Min => AggregateFunction::Min, + protobuf::AggregateFunction::Max => AggregateFunction::Max, + protobuf::AggregateFunction::Sum => AggregateFunction::Sum, + protobuf::AggregateFunction::Avg => AggregateFunction::Avg, + protobuf::AggregateFunction::Count => AggregateFunction::Count, + }; + + Ok(Expr::AggregateFunction { + fun, + args: vec![parse_required_expr(&expr.expr)?], + distinct: false, //TODO + }) + } + ExprType::Alias(alias) => Ok(Expr::Alias( + Box::new(parse_required_expr(&alias.expr)?), + alias.alias.clone(), + )), + ExprType::IsNullExpr(is_null) => { + Ok(Expr::IsNull(Box::new(parse_required_expr(&is_null.expr)?))) + } + ExprType::IsNotNullExpr(is_not_null) => Ok(Expr::IsNotNull(Box::new( + parse_required_expr(&is_not_null.expr)?, + ))), + ExprType::NotExpr(not) => Ok(Expr::Not(Box::new(parse_required_expr(¬.expr)?))), + ExprType::Between(between) => Ok(Expr::Between { + expr: Box::new(parse_required_expr(&between.expr)?), + negated: between.negated, + low: Box::new(parse_required_expr(&between.low)?), + high: Box::new(parse_required_expr(&between.high)?), + }), + ExprType::Case(case) => { + let when_then_expr = case + .when_then_expr + .iter() + .map(|e| { + Ok(( + Box::new(match &e.when_expr { + Some(e) => e.try_into(), + None => Err(proto_error("Missing required expression")), + }?), + Box::new(match &e.then_expr { + Some(e) => e.try_into(), + None => Err(proto_error("Missing required expression")), + }?), + )) + }) + .collect::, Box)>, BallistaError>>()?; + Ok(Expr::Case { + expr: parse_optional_expr(&case.expr)?.map(Box::new), + when_then_expr, + else_expr: parse_optional_expr(&case.else_expr)?.map(Box::new), + }) + } + ExprType::Cast(cast) => { + let expr = Box::new(parse_required_expr(&cast.expr)?); + let arrow_type: &protobuf::ArrowType = cast + .arrow_type + .as_ref() + .ok_or_else(|| proto_error("Protobuf deserialization error: CastNode message missing required field 'arrow_type'"))?; + let data_type = arrow_type.try_into()?; + Ok(Expr::Cast { expr, data_type }) + } + ExprType::Sort(sort) => Ok(Expr::Sort { + expr: Box::new(parse_required_expr(&sort.expr)?), + asc: sort.asc, + nulls_first: sort.nulls_first, + }), + ExprType::Negative(negative) => Ok(Expr::Negative(Box::new(parse_required_expr( + &negative.expr, + )?))), + ExprType::InList(in_list) => Ok(Expr::InList { + expr: Box::new(parse_required_expr(&in_list.expr)?), + list: in_list + .list + .iter() + .map(|expr| expr.try_into()) + .collect::, _>>()?, + negated: in_list.negated, + }), + ExprType::Wildcard(_) => Ok(Expr::Wildcard), + ExprType::ScalarFunction(expr) => { + let scalar_function = + protobuf::ScalarFunction::from_i32(expr.fun).ok_or_else(|| { + proto_error(format!("Received an unknown scalar function: {}", expr.fun)) + })?; + match scalar_function { + protobuf::ScalarFunction::Sqrt => Ok(sqrt((&expr.expr[0]).try_into()?)), + protobuf::ScalarFunction::Sin => Ok(sin((&expr.expr[0]).try_into()?)), + protobuf::ScalarFunction::Cos => Ok(cos((&expr.expr[0]).try_into()?)), + protobuf::ScalarFunction::Tan => Ok(tan((&expr.expr[0]).try_into()?)), + // protobuf::ScalarFunction::Asin => Ok(asin(&expr.expr[0]).try_into()?)), + // protobuf::ScalarFunction::Acos => Ok(acos(&expr.expr[0]).try_into()?)), + protobuf::ScalarFunction::Atan => Ok(atan((&expr.expr[0]).try_into()?)), + protobuf::ScalarFunction::Exp => Ok(exp((&expr.expr[0]).try_into()?)), + protobuf::ScalarFunction::Log2 => Ok(log2((&expr.expr[0]).try_into()?)), + protobuf::ScalarFunction::Log10 => Ok(log10((&expr.expr[0]).try_into()?)), + protobuf::ScalarFunction::Floor => Ok(floor((&expr.expr[0]).try_into()?)), + protobuf::ScalarFunction::Ceil => Ok(ceil((&expr.expr[0]).try_into()?)), + protobuf::ScalarFunction::Round => Ok(round((&expr.expr[0]).try_into()?)), + protobuf::ScalarFunction::Trunc => Ok(trunc((&expr.expr[0]).try_into()?)), + protobuf::ScalarFunction::Abs => Ok(abs((&expr.expr[0]).try_into()?)), + protobuf::ScalarFunction::Signum => Ok(signum((&expr.expr[0]).try_into()?)), + protobuf::ScalarFunction::Octetlength => { + Ok(length((&expr.expr[0]).try_into()?)) + } + // // protobuf::ScalarFunction::Concat => Ok(concat((&expr.expr[0]).try_into()?)), + protobuf::ScalarFunction::Lower => Ok(lower((&expr.expr[0]).try_into()?)), + protobuf::ScalarFunction::Upper => Ok(upper((&expr.expr[0]).try_into()?)), + protobuf::ScalarFunction::Trim => Ok(trim((&expr.expr[0]).try_into()?)), + protobuf::ScalarFunction::Ltrim => Ok(ltrim((&expr.expr[0]).try_into()?)), + protobuf::ScalarFunction::Rtrim => Ok(rtrim((&expr.expr[0]).try_into()?)), + // protobuf::ScalarFunction::Totimestamp => Ok(to_timestamp((&expr.expr[0]).try_into()?)), + // protobuf::ScalarFunction::Array => Ok(array((&expr.expr[0]).try_into()?)), + // // protobuf::ScalarFunction::Nullif => Ok(nulli((&expr.expr[0]).try_into()?)), + // protobuf::ScalarFunction::Datetrunc => Ok(date_trunc((&expr.expr[0]).try_into()?)), + // protobuf::ScalarFunction::Md5 => Ok(md5((&expr.expr[0]).try_into()?)), + protobuf::ScalarFunction::Sha224 => Ok(sha224((&expr.expr[0]).try_into()?)), + protobuf::ScalarFunction::Sha256 => Ok(sha256((&expr.expr[0]).try_into()?)), + protobuf::ScalarFunction::Sha384 => Ok(sha384((&expr.expr[0]).try_into()?)), + protobuf::ScalarFunction::Sha512 => Ok(sha512((&expr.expr[0]).try_into()?)), + _ => Err(proto_error( + "Protobuf deserialization error: Unsupported scalar function", + )), + } + } + } + } +} + +fn from_proto_binary_op(op: &str) -> Result { + match op { + "And" => Ok(Operator::And), + "Or" => Ok(Operator::Or), + "Eq" => Ok(Operator::Eq), + "NotEq" => Ok(Operator::NotEq), + "LtEq" => Ok(Operator::LtEq), + "Lt" => Ok(Operator::Lt), + "Gt" => Ok(Operator::Gt), + "GtEq" => Ok(Operator::GtEq), + "Plus" => Ok(Operator::Plus), + "Minus" => Ok(Operator::Minus), + "Multiply" => Ok(Operator::Multiply), + "Divide" => Ok(Operator::Divide), + "Like" => Ok(Operator::Like), + other => Err(proto_error(format!( + "Unsupported binary operator '{:?}'", + other + ))), + } +} + +impl TryInto for &protobuf::ScalarType { + type Error = BallistaError; + fn try_into(self) -> Result { + let pb_scalartype = self + .datatype + .as_ref() + .ok_or_else(|| proto_error("ScalarType message missing required field 'datatype'"))?; + pb_scalartype.try_into() + } +} + +impl TryInto for &protobuf::Schema { + type Error = BallistaError; + + fn try_into(self) -> Result { + let fields = self + .columns + .iter() + .map(|c| { + let pb_arrow_type_res = c + .arrow_type + .as_ref() + .ok_or_else(|| proto_error("Protobuf deserialization error: Field message was missing required field 'arrow_type'")); + let pb_arrow_type: &protobuf::ArrowType = match pb_arrow_type_res { + Ok(res) => res, + Err(e) => return Err(e), + }; + Ok(Field::new(&c.name, pb_arrow_type.try_into()?, c.nullable)) + }) + .collect::, _>>()?; + Ok(Schema::new(fields)) + } +} + +impl TryInto for &protobuf::Field { + type Error = BallistaError; + fn try_into(self) -> Result { + let pb_datatype = self.arrow_type.as_ref().ok_or_else(|| { + proto_error( + "Protobuf deserialization error: Field message missing required field 'arrow_type'", + ) + })?; + + Ok(arrow::datatypes::Field::new( + self.name.as_str(), + pb_datatype.as_ref().try_into()?, + self.nullable, + )) + } +} + +use datafusion::physical_plan::datetime_expressions::{date_trunc, to_timestamp}; +use datafusion::prelude::{ + array, length, lower, ltrim, md5, rtrim, sha224, sha256, sha384, sha512, trim, upper, +}; +use std::convert::TryFrom; + +impl TryFrom for protobuf::FileType { + type Error = BallistaError; + fn try_from(value: i32) -> Result { + use protobuf::FileType; + match value { + _x if _x == FileType::NdJson as i32 => Ok(FileType::NdJson), + _x if _x == FileType::Parquet as i32 => Ok(FileType::Parquet), + _x if _x == FileType::Csv as i32 => Ok(FileType::Csv), + invalid => Err(BallistaError::General(format!( + "Attempted to convert invalid i32 to protobuf::Filetype: {}", + invalid + ))), + } + } +} + +impl Into for protobuf::FileType { + fn into(self) -> datafusion::sql::parser::FileType { + use datafusion::sql::parser::FileType; + match self { + protobuf::FileType::NdJson => FileType::NdJson, + protobuf::FileType::Parquet => FileType::Parquet, + protobuf::FileType::Csv => FileType::CSV, + } + } +} + +fn parse_required_expr(p: &Option>) -> Result { + match p { + Some(expr) => expr.as_ref().try_into(), + None => Err(proto_error("Missing required expression")), + } +} + +fn parse_optional_expr( + p: &Option>, +) -> Result, BallistaError> { + match p { + Some(expr) => expr.as_ref().try_into().map(Some), + None => Ok(None), + } +} diff --git a/rust/ballista/rust/core/src/serde/logical_plan/mod.rs b/rust/ballista/rust/core/src/serde/logical_plan/mod.rs new file mode 100644 index 0000000000000..ba4590a697e6f --- /dev/null +++ b/rust/ballista/rust/core/src/serde/logical_plan/mod.rs @@ -0,0 +1,923 @@ +// 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. + +pub mod from_proto; +pub mod to_proto; + +#[cfg(test)] + +mod roundtrip_tests { + + use super::super::{super::error::Result, protobuf}; + use crate::error::BallistaError; + use arrow::datatypes::{DataType, Field, Schema}; + use core::panic; + use datafusion::physical_plan::functions::BuiltinScalarFunction::Sqrt; + use datafusion::{ + logical_plan::{Expr, LogicalPlan, LogicalPlanBuilder}, + physical_plan::csv::CsvReadOptions, + prelude::*, + scalar::ScalarValue, + }; + use protobuf::arrow_type; + use std::convert::TryInto; + + //Given a identity of a LogicalPlan converts it to protobuf and back, using debug formatting to test equality. + macro_rules! roundtrip_test { + ($initial_struct:ident, $proto_type:ty, $struct_type:ty) => { + let proto: $proto_type = (&$initial_struct).try_into()?; + + let round_trip: $struct_type = (&proto).try_into()?; + + assert_eq!( + format!("{:?}", $initial_struct), + format!("{:?}", round_trip) + ); + }; + ($initial_struct:ident, $struct_type:ty) => { + roundtrip_test!($initial_struct, protobuf::LogicalPlanNode, $struct_type); + }; + ($initial_struct:ident) => { + roundtrip_test!($initial_struct, protobuf::LogicalPlanNode, LogicalPlan); + }; + } + + #[test] + + fn roundtrip_repartition() -> Result<()> { + use datafusion::logical_plan::Partitioning; + + let test_batch_sizes = [usize::MIN, usize::MAX, 43256]; + + let test_expr: Vec = vec![ + Expr::Column("c1".to_string()) + Expr::Column("c2".to_string()), + Expr::Literal((4.0).into()), + ]; + + let schema = Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("first_name", DataType::Utf8, false), + Field::new("last_name", DataType::Utf8, false), + Field::new("state", DataType::Utf8, false), + Field::new("salary", DataType::Int32, false), + ]); + + let plan = std::sync::Arc::new( + LogicalPlanBuilder::scan_csv( + "employee.csv", + CsvReadOptions::new().schema(&schema).has_header(true), + Some(vec![3, 4]), + ) + .and_then(|plan| plan.sort(&[col("salary")])) + .and_then(|plan| plan.build()) + .map_err(BallistaError::DataFusionError)?, + ); + + for batch_size in test_batch_sizes.iter() { + let rr_repartition = Partitioning::RoundRobinBatch(*batch_size); + + let roundtrip_plan = LogicalPlan::Repartition { + input: plan.clone(), + partitioning_scheme: rr_repartition, + }; + + roundtrip_test!(roundtrip_plan); + + let h_repartition = Partitioning::Hash(test_expr.clone(), *batch_size); + + let roundtrip_plan = LogicalPlan::Repartition { + input: plan.clone(), + partitioning_scheme: h_repartition, + }; + + roundtrip_test!(roundtrip_plan); + + let no_expr_hrepartition = Partitioning::Hash(Vec::new(), *batch_size); + + let roundtrip_plan = LogicalPlan::Repartition { + input: plan.clone(), + partitioning_scheme: no_expr_hrepartition, + }; + + roundtrip_test!(roundtrip_plan); + } + + Ok(()) + } + + fn new_box_field(name: &str, dt: DataType, nullable: bool) -> Box { + Box::new(arrow::datatypes::Field::new(name, dt, nullable)) + } + + #[test] + fn scalar_values_error_serialization() -> Result<()> { + use arrow::datatypes::DataType; + use datafusion::scalar::ScalarValue; + let should_fail_on_seralize: Vec = vec![ + //Should fail due to inconsistent types + ScalarValue::List( + Some(vec![ + ScalarValue::Int16(None), + ScalarValue::Float32(Some(32.0)), + ]), + DataType::List(new_box_field("item", DataType::Int16, true)), + ), + ScalarValue::List( + Some(vec![ + ScalarValue::Float32(None), + ScalarValue::Float32(Some(32.0)), + ]), + DataType::List(new_box_field("item", DataType::Int16, true)), + ), + ScalarValue::List( + Some(vec![ + ScalarValue::List( + None, + DataType::List(new_box_field("level2", DataType::Float32, true)), + ), + ScalarValue::List( + Some(vec![ + ScalarValue::Float32(Some(-213.1)), + ScalarValue::Float32(None), + ScalarValue::Float32(Some(5.5)), + ScalarValue::Float32(Some(2.0)), + ScalarValue::Float32(Some(1.0)), + ]), + DataType::List(new_box_field("level2", DataType::Float32, true)), + ), + ScalarValue::List( + None, + DataType::List(new_box_field( + "lists are typed inconsistently", + DataType::Int16, + true, + )), + ), + ]), + DataType::List(new_box_field( + "level1", + DataType::List(new_box_field("level2", DataType::Float32, true)), + true, + )), + ), + ]; + + for test_case in should_fail_on_seralize.into_iter() { + let res: Result = (&test_case).try_into(); + if let Ok(val) = res { + return Err(BallistaError::General(format!( + "The value {:?} should not have been able to serialize. Serialized to :{:?}", + test_case, val + ))); + } + } + Ok(()) + } + + #[test] + fn round_trip_scalar_values() -> Result<()> { + use arrow::datatypes::DataType; + use datafusion::scalar::ScalarValue; + let should_pass: Vec = vec![ + ScalarValue::Boolean(None), + ScalarValue::Float32(None), + ScalarValue::Float64(None), + ScalarValue::Int8(None), + ScalarValue::Int16(None), + ScalarValue::Int32(None), + ScalarValue::Int64(None), + ScalarValue::UInt8(None), + ScalarValue::UInt16(None), + ScalarValue::UInt32(None), + ScalarValue::UInt64(None), + ScalarValue::Utf8(None), + ScalarValue::LargeUtf8(None), + ScalarValue::List(None, DataType::Boolean), + ScalarValue::Date32(None), + ScalarValue::TimeMicrosecond(None), + ScalarValue::TimeNanosecond(None), + ScalarValue::Boolean(Some(true)), + ScalarValue::Boolean(Some(false)), + ScalarValue::Float32(Some(1.0)), + ScalarValue::Float32(Some(f32::MAX)), + ScalarValue::Float32(Some(f32::MIN)), + ScalarValue::Float32(Some(-2000.0)), + ScalarValue::Float64(Some(1.0)), + ScalarValue::Float64(Some(f64::MAX)), + ScalarValue::Float64(Some(f64::MIN)), + ScalarValue::Float64(Some(-2000.0)), + ScalarValue::Int8(Some(i8::MIN)), + ScalarValue::Int8(Some(i8::MAX)), + ScalarValue::Int8(Some(0)), + ScalarValue::Int8(Some(-15)), + ScalarValue::Int16(Some(i16::MIN)), + ScalarValue::Int16(Some(i16::MAX)), + ScalarValue::Int16(Some(0)), + ScalarValue::Int16(Some(-15)), + ScalarValue::Int32(Some(i32::MIN)), + ScalarValue::Int32(Some(i32::MAX)), + ScalarValue::Int32(Some(0)), + ScalarValue::Int32(Some(-15)), + ScalarValue::Int64(Some(i64::MIN)), + ScalarValue::Int64(Some(i64::MAX)), + ScalarValue::Int64(Some(0)), + ScalarValue::Int64(Some(-15)), + ScalarValue::UInt8(Some(u8::MAX)), + ScalarValue::UInt8(Some(0)), + ScalarValue::UInt16(Some(u16::MAX)), + ScalarValue::UInt16(Some(0)), + ScalarValue::UInt32(Some(u32::MAX)), + ScalarValue::UInt32(Some(0)), + ScalarValue::UInt64(Some(u64::MAX)), + ScalarValue::UInt64(Some(0)), + ScalarValue::Utf8(Some(String::from("Test string "))), + ScalarValue::LargeUtf8(Some(String::from("Test Large utf8"))), + ScalarValue::Date32(Some(0)), + ScalarValue::Date32(Some(i32::MAX)), + ScalarValue::TimeNanosecond(Some(0)), + ScalarValue::TimeNanosecond(Some(i64::MAX)), + ScalarValue::TimeMicrosecond(Some(0)), + ScalarValue::TimeMicrosecond(Some(i64::MAX)), + ScalarValue::TimeMicrosecond(None), + ScalarValue::List( + Some(vec![ + ScalarValue::Float32(Some(-213.1)), + ScalarValue::Float32(None), + ScalarValue::Float32(Some(5.5)), + ScalarValue::Float32(Some(2.0)), + ScalarValue::Float32(Some(1.0)), + ]), + DataType::List(new_box_field("level1", DataType::Float32, true)), + ), + ScalarValue::List( + Some(vec![ + ScalarValue::List( + None, + DataType::List(new_box_field("level2", DataType::Float32, true)), + ), + ScalarValue::List( + Some(vec![ + ScalarValue::Float32(Some(-213.1)), + ScalarValue::Float32(None), + ScalarValue::Float32(Some(5.5)), + ScalarValue::Float32(Some(2.0)), + ScalarValue::Float32(Some(1.0)), + ]), + DataType::List(new_box_field("level2", DataType::Float32, true)), + ), + ]), + DataType::List(new_box_field( + "level1", + DataType::List(new_box_field("level2", DataType::Float32, true)), + true, + )), + ), + ]; + + for test_case in should_pass.into_iter() { + let proto: protobuf::ScalarValue = (&test_case).try_into()?; + let _roundtrip: ScalarValue = (&proto).try_into()?; + } + + Ok(()) + } + + #[test] + fn round_trip_scalar_types() -> Result<()> { + use arrow::datatypes::DataType; + use arrow::datatypes::{IntervalUnit, TimeUnit}; + let should_pass: Vec = vec![ + DataType::Boolean, + DataType::Int8, + DataType::Int16, + DataType::Int32, + DataType::Int64, + DataType::UInt8, + DataType::UInt16, + DataType::UInt32, + DataType::UInt64, + DataType::Float32, + DataType::Float64, + DataType::Date32, + DataType::Time64(TimeUnit::Microsecond), + DataType::Time64(TimeUnit::Nanosecond), + DataType::Utf8, + DataType::LargeUtf8, + //Recursive list tests + DataType::List(new_box_field("Level1", DataType::Boolean, true)), + DataType::List(new_box_field( + "Level1", + DataType::List(new_box_field("Level2", DataType::Date32, true)), + true, + )), + ]; + + let should_fail: Vec = vec![ + DataType::Null, + DataType::Float16, + //Add more timestamp tests + DataType::Timestamp(TimeUnit::Millisecond, None), + DataType::Date64, + DataType::Time32(TimeUnit::Second), + DataType::Time32(TimeUnit::Millisecond), + DataType::Time32(TimeUnit::Microsecond), + DataType::Time32(TimeUnit::Nanosecond), + DataType::Time64(TimeUnit::Second), + DataType::Time64(TimeUnit::Millisecond), + DataType::Duration(TimeUnit::Second), + DataType::Duration(TimeUnit::Millisecond), + DataType::Duration(TimeUnit::Microsecond), + DataType::Duration(TimeUnit::Nanosecond), + DataType::Interval(IntervalUnit::YearMonth), + DataType::Interval(IntervalUnit::DayTime), + DataType::Binary, + DataType::FixedSizeBinary(0), + DataType::FixedSizeBinary(1234), + DataType::FixedSizeBinary(-432), + DataType::LargeBinary, + DataType::Decimal(1345, 5431), + //Recursive list tests + DataType::List(new_box_field("Level1", DataType::Binary, true)), + DataType::List(new_box_field( + "Level1", + DataType::List(new_box_field( + "Level2", + DataType::FixedSizeBinary(53), + false, + )), + true, + )), + //Fixed size lists + DataType::FixedSizeList(new_box_field("Level1", DataType::Binary, true), 4), + DataType::FixedSizeList( + new_box_field( + "Level1", + DataType::List(new_box_field( + "Level2", + DataType::FixedSizeBinary(53), + false, + )), + true, + ), + 41, + ), + //Struct Testing + DataType::Struct(vec![ + Field::new("nullable", DataType::Boolean, false), + Field::new("name", DataType::Utf8, false), + Field::new("datatype", DataType::Binary, false), + ]), + DataType::Struct(vec![ + Field::new("nullable", DataType::Boolean, false), + Field::new("name", DataType::Utf8, false), + Field::new("datatype", DataType::Binary, false), + Field::new( + "nested_struct", + DataType::Struct(vec![ + Field::new("nullable", DataType::Boolean, false), + Field::new("name", DataType::Utf8, false), + Field::new("datatype", DataType::Binary, false), + ]), + true, + ), + ]), + DataType::Union(vec![ + Field::new("nullable", DataType::Boolean, false), + Field::new("name", DataType::Utf8, false), + Field::new("datatype", DataType::Binary, false), + ]), + DataType::Union(vec![ + Field::new("nullable", DataType::Boolean, false), + Field::new("name", DataType::Utf8, false), + Field::new("datatype", DataType::Binary, false), + Field::new( + "nested_struct", + DataType::Struct(vec![ + Field::new("nullable", DataType::Boolean, false), + Field::new("name", DataType::Utf8, false), + Field::new("datatype", DataType::Binary, false), + ]), + true, + ), + ]), + DataType::Dictionary( + Box::new(DataType::Utf8), + Box::new(DataType::Struct(vec![ + Field::new("nullable", DataType::Boolean, false), + Field::new("name", DataType::Utf8, false), + Field::new("datatype", DataType::Binary, false), + ])), + ), + DataType::Dictionary( + Box::new(DataType::Decimal(10, 50)), + Box::new(DataType::FixedSizeList( + new_box_field("Level1", DataType::Binary, true), + 4, + )), + ), + ]; + + for test_case in should_pass.into_iter() { + let proto: protobuf::ScalarType = (&test_case).try_into()?; + let roundtrip: DataType = (&proto).try_into()?; + assert_eq!(format!("{:?}", test_case), format!("{:?}", roundtrip)); + } + + let mut success: Vec = Vec::new(); + for test_case in should_fail.into_iter() { + let proto: Result = (&test_case).try_into(); + if proto.is_ok() { + success.push(test_case) + } + } + if !success.is_empty() { + return Err(BallistaError::General(format!( + "The following items which should have ressulted in an error completed successfully: {:?}", + success + ))); + } + Ok(()) + } + + #[test] + fn round_trip_datatype() -> Result<()> { + use arrow::datatypes::DataType; + use arrow::datatypes::{IntervalUnit, TimeUnit}; + let test_cases: Vec = vec![ + DataType::Null, + DataType::Boolean, + DataType::Int8, + DataType::Int16, + DataType::Int32, + DataType::Int64, + DataType::UInt8, + DataType::UInt16, + DataType::UInt32, + DataType::UInt64, + DataType::Float16, + DataType::Float32, + DataType::Float64, + //Add more timestamp tests + DataType::Timestamp(TimeUnit::Millisecond, None), + DataType::Date32, + DataType::Date64, + DataType::Time32(TimeUnit::Second), + DataType::Time32(TimeUnit::Millisecond), + DataType::Time32(TimeUnit::Microsecond), + DataType::Time32(TimeUnit::Nanosecond), + DataType::Time64(TimeUnit::Second), + DataType::Time64(TimeUnit::Millisecond), + DataType::Time64(TimeUnit::Microsecond), + DataType::Time64(TimeUnit::Nanosecond), + DataType::Duration(TimeUnit::Second), + DataType::Duration(TimeUnit::Millisecond), + DataType::Duration(TimeUnit::Microsecond), + DataType::Duration(TimeUnit::Nanosecond), + DataType::Interval(IntervalUnit::YearMonth), + DataType::Interval(IntervalUnit::DayTime), + DataType::Binary, + DataType::FixedSizeBinary(0), + DataType::FixedSizeBinary(1234), + DataType::FixedSizeBinary(-432), + DataType::LargeBinary, + DataType::Utf8, + DataType::LargeUtf8, + DataType::Decimal(1345, 5431), + //Recursive list tests + DataType::List(new_box_field("Level1", DataType::Binary, true)), + DataType::List(new_box_field( + "Level1", + DataType::List(new_box_field( + "Level2", + DataType::FixedSizeBinary(53), + false, + )), + true, + )), + //Fixed size lists + DataType::FixedSizeList(new_box_field("Level1", DataType::Binary, true), 4), + DataType::FixedSizeList( + new_box_field( + "Level1", + DataType::List(new_box_field( + "Level2", + DataType::FixedSizeBinary(53), + false, + )), + true, + ), + 41, + ), + //Struct Testing + DataType::Struct(vec![ + Field::new("nullable", DataType::Boolean, false), + Field::new("name", DataType::Utf8, false), + Field::new("datatype", DataType::Binary, false), + ]), + DataType::Struct(vec![ + Field::new("nullable", DataType::Boolean, false), + Field::new("name", DataType::Utf8, false), + Field::new("datatype", DataType::Binary, false), + Field::new( + "nested_struct", + DataType::Struct(vec![ + Field::new("nullable", DataType::Boolean, false), + Field::new("name", DataType::Utf8, false), + Field::new("datatype", DataType::Binary, false), + ]), + true, + ), + ]), + DataType::Union(vec![ + Field::new("nullable", DataType::Boolean, false), + Field::new("name", DataType::Utf8, false), + Field::new("datatype", DataType::Binary, false), + ]), + DataType::Union(vec![ + Field::new("nullable", DataType::Boolean, false), + Field::new("name", DataType::Utf8, false), + Field::new("datatype", DataType::Binary, false), + Field::new( + "nested_struct", + DataType::Struct(vec![ + Field::new("nullable", DataType::Boolean, false), + Field::new("name", DataType::Utf8, false), + Field::new("datatype", DataType::Binary, false), + ]), + true, + ), + ]), + DataType::Dictionary( + Box::new(DataType::Utf8), + Box::new(DataType::Struct(vec![ + Field::new("nullable", DataType::Boolean, false), + Field::new("name", DataType::Utf8, false), + Field::new("datatype", DataType::Binary, false), + ])), + ), + DataType::Dictionary( + Box::new(DataType::Decimal(10, 50)), + Box::new(DataType::FixedSizeList( + new_box_field("Level1", DataType::Binary, true), + 4, + )), + ), + ]; + + for test_case in test_cases.into_iter() { + let proto: protobuf::ArrowType = (&test_case).into(); + let roundtrip: DataType = (&proto).try_into()?; + assert_eq!(format!("{:?}", test_case), format!("{:?}", roundtrip)); + } + Ok(()) + } + + #[test] + fn roundtrip_null_scalar_values() -> Result<()> { + use arrow::datatypes::DataType; + use arrow::datatypes::Field; + use datafusion::scalar::ScalarValue; + let test_types = vec![ + ScalarValue::Boolean(None), + ScalarValue::Float32(None), + ScalarValue::Float64(None), + ScalarValue::Int8(None), + ScalarValue::Int16(None), + ScalarValue::Int32(None), + ScalarValue::Int64(None), + ScalarValue::UInt8(None), + ScalarValue::UInt16(None), + ScalarValue::UInt32(None), + ScalarValue::UInt64(None), + ScalarValue::Utf8(None), + ScalarValue::LargeUtf8(None), + ScalarValue::Date32(None), + ScalarValue::TimeMicrosecond(None), + ScalarValue::TimeNanosecond(None), + //ScalarValue::List(None, DataType::Boolean) + ]; + + for test_case in test_types.into_iter() { + let proto_scalar: protobuf::ScalarValue = (&test_case).try_into()?; + let returned_scalar: datafusion::scalar::ScalarValue = (&proto_scalar).try_into()?; + assert_eq!( + format!("{:?}", &test_case), + format!("{:?}", returned_scalar) + ); + } + + Ok(()) + } + + #[test] + + fn roundtrip_create_external_table() -> Result<()> { + let schema = Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("first_name", DataType::Utf8, false), + Field::new("last_name", DataType::Utf8, false), + Field::new("state", DataType::Utf8, false), + Field::new("salary", DataType::Int32, false), + ]); + + use datafusion::logical_plan::ToDFSchema; + + let df_schema_ref = schema.to_dfschema_ref()?; + + use datafusion::sql::parser::FileType; + + let filetypes: [FileType; 3] = [FileType::NdJson, FileType::Parquet, FileType::CSV]; + + for file in filetypes.iter() { + let create_table_node = LogicalPlan::CreateExternalTable { + schema: df_schema_ref.clone(), + name: String::from("TestName"), + location: String::from("employee.csv"), + file_type: *file, + has_header: true, + }; + + roundtrip_test!(create_table_node); + } + + Ok(()) + } + + #[test] + + fn roundtrip_explain() -> Result<()> { + let schema = Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("first_name", DataType::Utf8, false), + Field::new("last_name", DataType::Utf8, false), + Field::new("state", DataType::Utf8, false), + Field::new("salary", DataType::Int32, false), + ]); + + let verbose_plan = LogicalPlanBuilder::scan_csv( + "employee.csv", + CsvReadOptions::new().schema(&schema).has_header(true), + Some(vec![3, 4]), + ) + .and_then(|plan| plan.sort(&[col("salary")])) + .and_then(|plan| plan.explain(true)) + .and_then(|plan| plan.build()) + .map_err(BallistaError::DataFusionError)?; + + let plan = LogicalPlanBuilder::scan_csv( + "employee.csv", + CsvReadOptions::new().schema(&schema).has_header(true), + Some(vec![3, 4]), + ) + .and_then(|plan| plan.sort(&[col("salary")])) + .and_then(|plan| plan.explain(false)) + .and_then(|plan| plan.build()) + .map_err(BallistaError::DataFusionError)?; + + roundtrip_test!(plan); + + roundtrip_test!(verbose_plan); + + Ok(()) + } + + #[test] + fn roundtrip_join() -> Result<()> { + let schema = Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("first_name", DataType::Utf8, false), + Field::new("last_name", DataType::Utf8, false), + Field::new("state", DataType::Utf8, false), + Field::new("salary", DataType::Int32, false), + ]); + + let scan_plan = LogicalPlanBuilder::empty(false) + .build() + .map_err(BallistaError::DataFusionError)?; + let plan = LogicalPlanBuilder::scan_csv( + "employee.csv", + CsvReadOptions::new().schema(&schema).has_header(true), + Some(vec![3, 4]), + ) + .and_then(|plan| plan.join(&scan_plan, JoinType::Inner, &["id"], &["id"])) + .and_then(|plan| plan.build()) + .map_err(BallistaError::DataFusionError)?; + + roundtrip_test!(plan); + Ok(()) + } + + #[test] + fn roundtrip_sort() -> Result<()> { + let schema = Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("first_name", DataType::Utf8, false), + Field::new("last_name", DataType::Utf8, false), + Field::new("state", DataType::Utf8, false), + Field::new("salary", DataType::Int32, false), + ]); + + let plan = LogicalPlanBuilder::scan_csv( + "employee.csv", + CsvReadOptions::new().schema(&schema).has_header(true), + Some(vec![3, 4]), + ) + .and_then(|plan| plan.sort(&[col("salary")])) + .and_then(|plan| plan.build()) + .map_err(BallistaError::DataFusionError)?; + roundtrip_test!(plan); + + Ok(()) + } + + #[test] + + fn roundtrip_empty_relation() -> Result<()> { + let plan_false = LogicalPlanBuilder::empty(false) + .build() + .map_err(BallistaError::DataFusionError)?; + + roundtrip_test!(plan_false); + + let plan_true = LogicalPlanBuilder::empty(true) + .build() + .map_err(BallistaError::DataFusionError)?; + + roundtrip_test!(plan_true); + + Ok(()) + } + + #[test] + + fn roundtrip_logical_plan() -> Result<()> { + let schema = Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("first_name", DataType::Utf8, false), + Field::new("last_name", DataType::Utf8, false), + Field::new("state", DataType::Utf8, false), + Field::new("salary", DataType::Int32, false), + ]); + + let plan = LogicalPlanBuilder::scan_csv( + "employee.csv", + CsvReadOptions::new().schema(&schema).has_header(true), + Some(vec![3, 4]), + ) + .and_then(|plan| plan.aggregate(&[col("state")], &[max(col("salary"))])) + .and_then(|plan| plan.build()) + .map_err(BallistaError::DataFusionError)?; + + roundtrip_test!(plan); + + Ok(()) + } + + #[test] + + fn roundtrip_not() -> Result<()> { + let test_expr = Expr::Not(Box::new(Expr::Literal((1.0).into()))); + + roundtrip_test!(test_expr, protobuf::LogicalExprNode, Expr); + + Ok(()) + } + + #[test] + + fn roundtrip_is_null() -> Result<()> { + let test_expr = Expr::IsNull(Box::new(Expr::Column("id".into()))); + + roundtrip_test!(test_expr, protobuf::LogicalExprNode, Expr); + + Ok(()) + } + + #[test] + + fn roundtrip_is_not_null() -> Result<()> { + let test_expr = Expr::IsNotNull(Box::new(Expr::Column("id".into()))); + + roundtrip_test!(test_expr, protobuf::LogicalExprNode, Expr); + + Ok(()) + } + + #[test] + + fn roundtrip_between() -> Result<()> { + let test_expr = Expr::Between { + expr: Box::new(Expr::Literal((1.0).into())), + negated: true, + low: Box::new(Expr::Literal((2.0).into())), + high: Box::new(Expr::Literal((3.0).into())), + }; + + roundtrip_test!(test_expr, protobuf::LogicalExprNode, Expr); + + Ok(()) + } + + #[test] + + fn roundtrip_case() -> Result<()> { + let test_expr = Expr::Case { + expr: Some(Box::new(Expr::Literal((1.0).into()))), + when_then_expr: vec![( + Box::new(Expr::Literal((2.0).into())), + Box::new(Expr::Literal((3.0).into())), + )], + else_expr: Some(Box::new(Expr::Literal((4.0).into()))), + }; + + roundtrip_test!(test_expr, protobuf::LogicalExprNode, Expr); + + Ok(()) + } + + #[test] + + fn roundtrip_cast() -> Result<()> { + let test_expr = Expr::Cast { + expr: Box::new(Expr::Literal((1.0).into())), + data_type: DataType::Boolean, + }; + + roundtrip_test!(test_expr, protobuf::LogicalExprNode, Expr); + + Ok(()) + } + + #[test] + + fn roundtrip_sort_expr() -> Result<()> { + let test_expr = Expr::Sort { + expr: Box::new(Expr::Literal((1.0).into())), + asc: true, + nulls_first: true, + }; + + roundtrip_test!(test_expr, protobuf::LogicalExprNode, Expr); + + Ok(()) + } + + #[test] + + fn roundtrip_negative() -> Result<()> { + let test_expr = Expr::Negative(Box::new(Expr::Literal((1.0).into()))); + + roundtrip_test!(test_expr, protobuf::LogicalExprNode, Expr); + + Ok(()) + } + + #[test] + + fn roundtrip_inlist() -> Result<()> { + let test_expr = Expr::InList { + expr: Box::new(Expr::Literal((1.0).into())), + list: vec![Expr::Literal((2.0).into())], + negated: true, + }; + + roundtrip_test!(test_expr, protobuf::LogicalExprNode, Expr); + + Ok(()) + } + + #[test] + + fn roundtrip_wildcard() -> Result<()> { + let test_expr = Expr::Wildcard; + + roundtrip_test!(test_expr, protobuf::LogicalExprNode, Expr); + + Ok(()) + } + + #[test] + fn roundtrip_sqrt() -> Result<()> { + let test_expr = Expr::ScalarFunction { + fun: Sqrt, + args: vec![col("col")], + }; + roundtrip_test!(test_expr, protobuf::LogicalExprNode, Expr); + + Ok(()) + } +} diff --git a/rust/ballista/rust/core/src/serde/logical_plan/to_proto.rs b/rust/ballista/rust/core/src/serde/logical_plan/to_proto.rs new file mode 100644 index 0000000000000..beee4182ffa70 --- /dev/null +++ b/rust/ballista/rust/core/src/serde/logical_plan/to_proto.rs @@ -0,0 +1,1209 @@ +// 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. + +//! Serde code to convert Arrow schemas and DataFusion logical plans to Ballista protocol +//! buffer format, allowing DataFusion logical plans to be serialized and transmitted between +//! processes. + +use std::{ + boxed, + convert::{TryFrom, TryInto}, +}; + +use crate::datasource::DFTableAdapter; +use crate::serde::{protobuf, BallistaError}; + +use arrow::datatypes::{DataType, Schema}; +use datafusion::datasource::CsvFile; +use datafusion::logical_plan::{Expr, JoinType, LogicalPlan}; +use datafusion::physical_plan::aggregates::AggregateFunction; +use datafusion::{datasource::parquet::ParquetTable, logical_plan::exprlist_to_fields}; +use protobuf::{ + arrow_type, logical_expr_node::ExprType, scalar_type, DateUnit, Field, PrimitiveScalarType, + ScalarListValue, ScalarType, +}; + +use super::super::proto_error; +use datafusion::physical_plan::functions::BuiltinScalarFunction; + +impl protobuf::IntervalUnit { + pub fn from_arrow_interval_unit(interval_unit: &arrow::datatypes::IntervalUnit) -> Self { + match interval_unit { + arrow::datatypes::IntervalUnit::YearMonth => protobuf::IntervalUnit::YearMonth, + arrow::datatypes::IntervalUnit::DayTime => protobuf::IntervalUnit::DayTime, + } + } + + pub fn from_i32_to_arrow( + interval_unit_i32: i32, + ) -> Result { + let pb_interval_unit = protobuf::IntervalUnit::from_i32(interval_unit_i32); + use arrow::datatypes::IntervalUnit; + match pb_interval_unit { + Some(interval_unit) => Ok(match interval_unit { + protobuf::IntervalUnit::YearMonth => IntervalUnit::YearMonth, + protobuf::IntervalUnit::DayTime => IntervalUnit::DayTime, + }), + None => Err(proto_error( + "Error converting i32 to DateUnit: Passed invalid variant", + )), + } + } +} +/* Arrow changed dates to no longer have date unit + +impl protobuf::DateUnit { + pub fn from_arrow_date_unit(val: &arrow::datatypes::DateUnit) -> Self { + match val { + arrow::datatypes::DateUnit::Day => protobuf::DateUnit::Day, + arrow::datatypes::DateUnit::Millisecond => protobuf::DateUnit::DateMillisecond, + } + } + pub fn from_i32_to_arrow(date_unit_i32: i32) -> Result { + let pb_date_unit = protobuf::DateUnit::from_i32(date_unit_i32); + use arrow::datatypes::DateUnit; + match pb_date_unit { + Some(date_unit) => Ok(match date_unit { + protobuf::DateUnit::Day => DateUnit::Day, + protobuf::DateUnit::DateMillisecond => DateUnit::Millisecond, + }), + None => Err(proto_error("Error converting i32 to DateUnit: Passed invalid variant")), + } + } + +}*/ + +impl protobuf::TimeUnit { + pub fn from_arrow_time_unit(val: &arrow::datatypes::TimeUnit) -> Self { + match val { + arrow::datatypes::TimeUnit::Second => protobuf::TimeUnit::Second, + arrow::datatypes::TimeUnit::Millisecond => protobuf::TimeUnit::TimeMillisecond, + arrow::datatypes::TimeUnit::Microsecond => protobuf::TimeUnit::Microsecond, + arrow::datatypes::TimeUnit::Nanosecond => protobuf::TimeUnit::Nanosecond, + } + } + pub fn from_i32_to_arrow( + time_unit_i32: i32, + ) -> Result { + let pb_time_unit = protobuf::TimeUnit::from_i32(time_unit_i32); + use arrow::datatypes::TimeUnit; + match pb_time_unit { + Some(time_unit) => Ok(match time_unit { + protobuf::TimeUnit::Second => TimeUnit::Second, + protobuf::TimeUnit::TimeMillisecond => TimeUnit::Millisecond, + protobuf::TimeUnit::Microsecond => TimeUnit::Microsecond, + protobuf::TimeUnit::Nanosecond => TimeUnit::Nanosecond, + }), + None => Err(proto_error( + "Error converting i32 to TimeUnit: Passed invalid variant", + )), + } + } +} + +impl From<&arrow::datatypes::Field> for protobuf::Field { + fn from(field: &arrow::datatypes::Field) -> Self { + protobuf::Field { + name: field.name().to_owned(), + arrow_type: Some(Box::new(field.data_type().into())), + nullable: field.is_nullable(), + children: Vec::new(), + } + } +} + +impl From<&arrow::datatypes::DataType> for protobuf::ArrowType { + fn from(val: &arrow::datatypes::DataType) -> protobuf::ArrowType { + protobuf::ArrowType { + arrow_type_enum: Some(val.into()), + } + } +} + +impl TryInto for &protobuf::ArrowType { + type Error = BallistaError; + fn try_into(self) -> Result { + let pb_arrow_type = self.arrow_type_enum.as_ref().ok_or_else(|| { + proto_error( + "Protobuf deserialization error: ArrowType missing required field 'data_type'", + ) + })?; + use arrow::datatypes::DataType; + Ok(match pb_arrow_type { + protobuf::arrow_type::ArrowTypeEnum::None(_) => DataType::Null, + protobuf::arrow_type::ArrowTypeEnum::Bool(_) => DataType::Boolean, + protobuf::arrow_type::ArrowTypeEnum::Uint8(_) => DataType::UInt8, + protobuf::arrow_type::ArrowTypeEnum::Int8(_) => DataType::Int8, + protobuf::arrow_type::ArrowTypeEnum::Uint16(_) => DataType::UInt16, + protobuf::arrow_type::ArrowTypeEnum::Int16(_) => DataType::Int16, + protobuf::arrow_type::ArrowTypeEnum::Uint32(_) => DataType::UInt32, + protobuf::arrow_type::ArrowTypeEnum::Int32(_) => DataType::Int32, + protobuf::arrow_type::ArrowTypeEnum::Uint64(_) => DataType::UInt64, + protobuf::arrow_type::ArrowTypeEnum::Int64(_) => DataType::Int64, + protobuf::arrow_type::ArrowTypeEnum::Float16(_) => DataType::Float16, + protobuf::arrow_type::ArrowTypeEnum::Float32(_) => DataType::Float32, + protobuf::arrow_type::ArrowTypeEnum::Float64(_) => DataType::Float64, + protobuf::arrow_type::ArrowTypeEnum::Utf8(_) => DataType::Utf8, + protobuf::arrow_type::ArrowTypeEnum::LargeUtf8(_) => DataType::LargeUtf8, + protobuf::arrow_type::ArrowTypeEnum::Binary(_) => DataType::Binary, + protobuf::arrow_type::ArrowTypeEnum::FixedSizeBinary(size) => { + DataType::FixedSizeBinary(*size) + } + protobuf::arrow_type::ArrowTypeEnum::LargeBinary(_) => DataType::LargeBinary, + protobuf::arrow_type::ArrowTypeEnum::Date32(_) => DataType::Date32, + protobuf::arrow_type::ArrowTypeEnum::Date64(_) => DataType::Date64, + protobuf::arrow_type::ArrowTypeEnum::Duration(time_unit_i32) => { + DataType::Duration(protobuf::TimeUnit::from_i32_to_arrow(*time_unit_i32)?) + } + protobuf::arrow_type::ArrowTypeEnum::Timestamp(timestamp) => DataType::Timestamp( + protobuf::TimeUnit::from_i32_to_arrow(timestamp.time_unit)?, + match timestamp.timezone.is_empty() { + true => None, + false => Some(timestamp.timezone.to_owned()), + }, + ), + protobuf::arrow_type::ArrowTypeEnum::Time32(time_unit_i32) => { + DataType::Time32(protobuf::TimeUnit::from_i32_to_arrow(*time_unit_i32)?) + } + protobuf::arrow_type::ArrowTypeEnum::Time64(time_unit_i32) => { + DataType::Time64(protobuf::TimeUnit::from_i32_to_arrow(*time_unit_i32)?) + } + protobuf::arrow_type::ArrowTypeEnum::Interval(interval_unit_i32) => DataType::Interval( + protobuf::IntervalUnit::from_i32_to_arrow(*interval_unit_i32)?, + ), + protobuf::arrow_type::ArrowTypeEnum::Decimal(protobuf::Decimal { + whole, + fractional, + }) => DataType::Decimal(*whole as usize, *fractional as usize), + protobuf::arrow_type::ArrowTypeEnum::List(boxed_list) => { + let field_ref = boxed_list + .field_type + .as_ref() + .ok_or_else(|| proto_error("Protobuf deserialization error: List message was missing required field 'field_type'"))? + .as_ref(); + arrow::datatypes::DataType::List(Box::new(field_ref.try_into()?)) + } + protobuf::arrow_type::ArrowTypeEnum::LargeList(boxed_list) => { + let field_ref = boxed_list + .field_type + .as_ref() + .ok_or_else(|| proto_error("Protobuf deserialization error: List message was missing required field 'field_type'"))? + .as_ref(); + arrow::datatypes::DataType::LargeList(Box::new(field_ref.try_into()?)) + } + protobuf::arrow_type::ArrowTypeEnum::FixedSizeList(boxed_list) => { + let fsl_ref = boxed_list.as_ref(); + let pb_fieldtype = fsl_ref + .field_type + .as_ref() + .ok_or_else(|| proto_error("Protobuf deserialization error: FixedSizeList message was missing required field 'field_type'"))?; + arrow::datatypes::DataType::FixedSizeList( + Box::new(pb_fieldtype.as_ref().try_into()?), + fsl_ref.list_size, + ) + } + protobuf::arrow_type::ArrowTypeEnum::Struct(struct_type) => { + let fields = struct_type + .sub_field_types + .iter() + .map(|field| field.try_into()) + .collect::, _>>()?; + arrow::datatypes::DataType::Struct(fields) + } + protobuf::arrow_type::ArrowTypeEnum::Union(union) => { + let union_types = union + .union_types + .iter() + .map(|field| field.try_into()) + .collect::, _>>()?; + arrow::datatypes::DataType::Union(union_types) + } + protobuf::arrow_type::ArrowTypeEnum::Dictionary(boxed_dict) => { + let dict_ref = boxed_dict.as_ref(); + let pb_key = dict_ref + .key + .as_ref() + .ok_or_else(|| proto_error("Protobuf deserialization error: Dictionary message was missing required field 'key'"))?; + let pb_value = dict_ref + .value + .as_ref() + .ok_or_else(|| proto_error("Protobuf deserialization error: Dictionary message was missing required field 'value'"))?; + arrow::datatypes::DataType::Dictionary( + Box::new(pb_key.as_ref().try_into()?), + Box::new(pb_value.as_ref().try_into()?), + ) + } + }) + } +} + +impl TryInto for &Box { + type Error = BallistaError; + fn try_into(self) -> Result { + let list_ref = self.as_ref(); + match &list_ref.field_type { + Some(pb_field) => { + let pb_field_ref = pb_field.as_ref(); + let arrow_field: arrow::datatypes::Field = pb_field_ref.try_into()?; + Ok(arrow::datatypes::DataType::List(Box::new(arrow_field))) + } + None => Err(proto_error( + "List message missing required field 'field_type'", + )), + } + } +} + +impl From<&arrow::datatypes::DataType> for protobuf::arrow_type::ArrowTypeEnum { + fn from(val: &arrow::datatypes::DataType) -> protobuf::arrow_type::ArrowTypeEnum { + use protobuf::arrow_type::ArrowTypeEnum; + use protobuf::ArrowType; + use protobuf::EmptyMessage; + match val { + DataType::Null => ArrowTypeEnum::None(EmptyMessage {}), + DataType::Boolean => ArrowTypeEnum::Bool(EmptyMessage {}), + DataType::Int8 => ArrowTypeEnum::Int8(EmptyMessage {}), + DataType::Int16 => ArrowTypeEnum::Int16(EmptyMessage {}), + DataType::Int32 => ArrowTypeEnum::Int32(EmptyMessage {}), + DataType::Int64 => ArrowTypeEnum::Int64(EmptyMessage {}), + DataType::UInt8 => ArrowTypeEnum::Uint8(EmptyMessage {}), + DataType::UInt16 => ArrowTypeEnum::Uint16(EmptyMessage {}), + DataType::UInt32 => ArrowTypeEnum::Uint32(EmptyMessage {}), + DataType::UInt64 => ArrowTypeEnum::Uint64(EmptyMessage {}), + DataType::Float16 => ArrowTypeEnum::Float16(EmptyMessage {}), + DataType::Float32 => ArrowTypeEnum::Float32(EmptyMessage {}), + DataType::Float64 => ArrowTypeEnum::Float64(EmptyMessage {}), + DataType::Timestamp(time_unit, timezone) => { + ArrowTypeEnum::Timestamp(protobuf::Timestamp { + time_unit: protobuf::TimeUnit::from_arrow_time_unit(time_unit) as i32, + timezone: timezone.to_owned().unwrap_or_else(String::new), + }) + } + DataType::Date32 => ArrowTypeEnum::Date32(EmptyMessage {}), + DataType::Date64 => ArrowTypeEnum::Date64(EmptyMessage {}), + DataType::Time32(time_unit) => { + ArrowTypeEnum::Time32(protobuf::TimeUnit::from_arrow_time_unit(time_unit) as i32) + } + DataType::Time64(time_unit) => { + ArrowTypeEnum::Time64(protobuf::TimeUnit::from_arrow_time_unit(time_unit) as i32) + } + DataType::Duration(time_unit) => { + ArrowTypeEnum::Duration(protobuf::TimeUnit::from_arrow_time_unit(time_unit) as i32) + } + DataType::Interval(interval_unit) => ArrowTypeEnum::Interval( + protobuf::IntervalUnit::from_arrow_interval_unit(interval_unit) as i32, + ), + DataType::Binary => ArrowTypeEnum::Binary(EmptyMessage {}), + DataType::FixedSizeBinary(size) => ArrowTypeEnum::FixedSizeBinary(*size), + DataType::LargeBinary => ArrowTypeEnum::LargeBinary(EmptyMessage {}), + DataType::Utf8 => ArrowTypeEnum::Utf8(EmptyMessage {}), + DataType::LargeUtf8 => ArrowTypeEnum::LargeUtf8(EmptyMessage {}), + DataType::List(item_type) => ArrowTypeEnum::List(Box::new(protobuf::List { + field_type: Some(Box::new(item_type.as_ref().into())), + })), + DataType::FixedSizeList(item_type, size) => { + ArrowTypeEnum::FixedSizeList(Box::new(protobuf::FixedSizeList { + field_type: Some(Box::new(item_type.as_ref().into())), + list_size: *size, + })) + } + DataType::LargeList(item_type) => ArrowTypeEnum::LargeList(Box::new(protobuf::List { + field_type: Some(Box::new(item_type.as_ref().into())), + })), + DataType::Struct(struct_fields) => ArrowTypeEnum::Struct(protobuf::Struct { + sub_field_types: struct_fields + .iter() + .map(|field| field.into()) + .collect::>(), + }), + DataType::Union(union_types) => ArrowTypeEnum::Union(protobuf::Union { + union_types: union_types + .iter() + .map(|field| field.into()) + .collect::>(), + }), + DataType::Dictionary(key_type, value_type) => { + ArrowTypeEnum::Dictionary(Box::new(protobuf::Dictionary { + key: Some(Box::new(key_type.as_ref().into())), + value: Some(Box::new(value_type.as_ref().into())), + })) + } + DataType::Decimal(whole, fractional) => ArrowTypeEnum::Decimal(protobuf::Decimal { + whole: *whole as u64, + fractional: *fractional as u64, + }), + } + } +} + +//Does not check if list subtypes are valid +fn is_valid_scalar_type_no_list_check(datatype: &arrow::datatypes::DataType) -> bool { + match datatype { + DataType::Boolean + | DataType::Int8 + | DataType::Int16 + | DataType::Int32 + | DataType::Int64 + | DataType::UInt8 + | DataType::UInt16 + | DataType::UInt32 + | DataType::UInt64 + | DataType::Float32 + | DataType::Float64 + | DataType::LargeUtf8 + | DataType::Utf8 + | DataType::Date32 => true, + DataType::Time64(time_unit) => matches!( + time_unit, + arrow::datatypes::TimeUnit::Microsecond | arrow::datatypes::TimeUnit::Nanosecond + ), + + DataType::List(_) => true, + _ => false, + } +} + +impl TryFrom<&arrow::datatypes::DataType> for protobuf::scalar_type::Datatype { + type Error = BallistaError; + fn try_from(val: &arrow::datatypes::DataType) -> Result { + use protobuf::scalar_type; + use protobuf::Field; + use protobuf::{List, PrimitiveScalarType}; + let scalar_value = match val { + DataType::Boolean => scalar_type::Datatype::Scalar(PrimitiveScalarType::Bool as i32), + DataType::Int8 => scalar_type::Datatype::Scalar(PrimitiveScalarType::Int8 as i32), + DataType::Int16 => scalar_type::Datatype::Scalar(PrimitiveScalarType::Int16 as i32), + DataType::Int32 => scalar_type::Datatype::Scalar(PrimitiveScalarType::Int32 as i32), + DataType::Int64 => scalar_type::Datatype::Scalar(PrimitiveScalarType::Int64 as i32), + DataType::UInt8 => scalar_type::Datatype::Scalar(PrimitiveScalarType::Uint8 as i32), + DataType::UInt16 => scalar_type::Datatype::Scalar(PrimitiveScalarType::Uint16 as i32), + DataType::UInt32 => scalar_type::Datatype::Scalar(PrimitiveScalarType::Uint32 as i32), + DataType::UInt64 => scalar_type::Datatype::Scalar(PrimitiveScalarType::Uint64 as i32), + DataType::Float32 => scalar_type::Datatype::Scalar(PrimitiveScalarType::Float32 as i32), + DataType::Float64 => scalar_type::Datatype::Scalar(PrimitiveScalarType::Float64 as i32), + DataType::Date32 => scalar_type::Datatype::Scalar(PrimitiveScalarType::Date32 as i32), + DataType::Time64(time_unit) => match time_unit { + arrow::datatypes::TimeUnit::Microsecond => scalar_type::Datatype::Scalar(PrimitiveScalarType::TimeMicrosecond as i32), + arrow::datatypes::TimeUnit::Nanosecond => scalar_type::Datatype::Scalar(PrimitiveScalarType::TimeNanosecond as i32), + _ => { + return Err(proto_error(format!( + "Found invalid time unit for scalar value, only TimeUnit::Microsecond and TimeUnit::Nanosecond are valid time units: {:?}", + time_unit + ))) + } + }, + DataType::Utf8 => scalar_type::Datatype::Scalar(PrimitiveScalarType::Utf8 as i32), + DataType::LargeUtf8 => scalar_type::Datatype::Scalar(PrimitiveScalarType::LargeUtf8 as i32), + DataType::List(field_type) => { + let mut field_names: Vec = Vec::new(); + let mut curr_field: &arrow::datatypes::Field = field_type.as_ref(); + field_names.push(curr_field.name().to_owned()); + //For each nested field check nested datatype, since datafusion scalars only support recursive lists with a leaf scalar type + // any other compound types are errors. + + while let DataType::List(nested_field_type) = curr_field.data_type() { + curr_field = nested_field_type.as_ref(); + field_names.push(curr_field.name().to_owned()); + if !is_valid_scalar_type_no_list_check(curr_field.data_type()) { + return Err(proto_error(format!("{:?} is an invalid scalar type", curr_field))); + } + } + let deepest_datatype = curr_field.data_type(); + if !is_valid_scalar_type_no_list_check(deepest_datatype) { + return Err(proto_error(format!("The list nested type {:?} is an invalid scalar type", curr_field))); + } + let pb_deepest_type: PrimitiveScalarType = match deepest_datatype { + DataType::Boolean => PrimitiveScalarType::Bool, + DataType::Int8 => PrimitiveScalarType::Int8, + DataType::Int16 => PrimitiveScalarType::Int16, + DataType::Int32 => PrimitiveScalarType::Int32, + DataType::Int64 => PrimitiveScalarType::Int64, + DataType::UInt8 => PrimitiveScalarType::Uint8, + DataType::UInt16 => PrimitiveScalarType::Uint16, + DataType::UInt32 => PrimitiveScalarType::Uint32, + DataType::UInt64 => PrimitiveScalarType::Uint64, + DataType::Float32 => PrimitiveScalarType::Float32, + DataType::Float64 => PrimitiveScalarType::Float64, + DataType::Date32 => PrimitiveScalarType::Date32, + DataType::Time64(time_unit) => match time_unit { + arrow::datatypes::TimeUnit::Microsecond => PrimitiveScalarType::TimeMicrosecond, + arrow::datatypes::TimeUnit::Nanosecond => PrimitiveScalarType::TimeNanosecond, + _ => { + return Err(proto_error(format!( + "Found invalid time unit for scalar value, only TimeUnit::Microsecond and TimeUnit::Nanosecond are valid time units: {:?}", + time_unit + ))) + } + }, + + DataType::Utf8 => PrimitiveScalarType::Utf8, + DataType::LargeUtf8 => PrimitiveScalarType::LargeUtf8, + _ => { + return Err(proto_error(format!( + "Error converting to Datatype to scalar type, {:?} is invalid as a datafusion scalar.", + val + ))) + } + }; + protobuf::scalar_type::Datatype::List(protobuf::ScalarListType { + field_names, + deepest_type: pb_deepest_type as i32, + }) + } + DataType::Null + | DataType::Float16 + | DataType::Timestamp(_, _) + | DataType::Date64 + | DataType::Time32(_) + | DataType::Duration(_) + | DataType::Interval(_) + | DataType::Binary + | DataType::FixedSizeBinary(_) + | DataType::LargeBinary + | DataType::FixedSizeList(_, _) + | DataType::LargeList(_) + | DataType::Struct(_) + | DataType::Union(_) + | DataType::Dictionary(_, _) + | DataType::Decimal(_, _) => { + return Err(proto_error(format!( + "Error converting to Datatype to scalar type, {:?} is invalid as a datafusion scalar.", + val + ))) + } + }; + Ok(scalar_value) + } +} + +impl TryFrom<&datafusion::scalar::ScalarValue> for protobuf::ScalarValue { + type Error = BallistaError; + fn try_from( + val: &datafusion::scalar::ScalarValue, + ) -> Result { + use datafusion::scalar; + use protobuf::scalar_value::Value; + use protobuf::PrimitiveScalarType; + let scalar_val = match val { + scalar::ScalarValue::Boolean(val) => { + create_proto_scalar(val, PrimitiveScalarType::Bool, |s| Value::BoolValue(*s)) + } + scalar::ScalarValue::Float32(val) => { + create_proto_scalar(val, PrimitiveScalarType::Float32, |s| { + Value::Float32Value(*s) + }) + } + scalar::ScalarValue::Float64(val) => { + create_proto_scalar(val, PrimitiveScalarType::Float64, |s| { + Value::Float64Value(*s) + }) + } + scalar::ScalarValue::Int8(val) => { + create_proto_scalar(val, PrimitiveScalarType::Int8, |s| { + Value::Int8Value(*s as i32) + }) + } + scalar::ScalarValue::Int16(val) => { + create_proto_scalar(val, PrimitiveScalarType::Int16, |s| { + Value::Int16Value(*s as i32) + }) + } + scalar::ScalarValue::Int32(val) => { + create_proto_scalar(val, PrimitiveScalarType::Int32, |s| Value::Int32Value(*s)) + } + scalar::ScalarValue::Int64(val) => { + create_proto_scalar(val, PrimitiveScalarType::Int64, |s| Value::Int64Value(*s)) + } + scalar::ScalarValue::UInt8(val) => { + create_proto_scalar(val, PrimitiveScalarType::Uint8, |s| { + Value::Uint8Value(*s as u32) + }) + } + scalar::ScalarValue::UInt16(val) => { + create_proto_scalar(val, PrimitiveScalarType::Uint16, |s| { + Value::Uint16Value(*s as u32) + }) + } + scalar::ScalarValue::UInt32(val) => { + create_proto_scalar(val, PrimitiveScalarType::Uint32, |s| Value::Uint32Value(*s)) + } + scalar::ScalarValue::UInt64(val) => { + create_proto_scalar(val, PrimitiveScalarType::Uint64, |s| Value::Uint64Value(*s)) + } + scalar::ScalarValue::Utf8(val) => { + create_proto_scalar(val, PrimitiveScalarType::Utf8, |s| { + Value::Utf8Value(s.to_owned()) + }) + } + scalar::ScalarValue::LargeUtf8(val) => { + create_proto_scalar(val, PrimitiveScalarType::LargeUtf8, |s| { + Value::LargeUtf8Value(s.to_owned()) + }) + } + scalar::ScalarValue::List(value, datatype) => { + println!("Current datatype of list: {:?}", datatype); + match value { + Some(values) => { + if values.is_empty() { + protobuf::ScalarValue { + value: Some(protobuf::scalar_value::Value::ListValue( + protobuf::ScalarListValue { + datatype: Some(datatype.try_into()?), + values: Vec::new(), + }, + )), + } + } else { + let scalar_type = match datatype { + DataType::List(field) => field.as_ref().data_type(), + _ => todo!("Proper error handling"), + }; + println!("Current scalar type for list: {:?}", scalar_type); + let type_checked_values: Vec = values + .iter() + .map(|scalar| match (scalar, scalar_type) { + (scalar::ScalarValue::List(_, arrow::datatypes::DataType::List(list_field)), arrow::datatypes::DataType::List(field)) => { + let scalar_datatype = field.data_type(); + let list_datatype = list_field.data_type(); + if std::mem::discriminant(list_datatype) != std::mem::discriminant(scalar_datatype) { + return Err(proto_error(format!( + "Protobuf serialization error: Lists with inconsistent typing {:?} and {:?} found within list", + list_datatype, scalar_datatype + ))); + } + scalar.try_into() + } + (scalar::ScalarValue::Boolean(_), arrow::datatypes::DataType::Boolean) => scalar.try_into(), + (scalar::ScalarValue::Float32(_), arrow::datatypes::DataType::Float32) => scalar.try_into(), + (scalar::ScalarValue::Float64(_), arrow::datatypes::DataType::Float64) => scalar.try_into(), + (scalar::ScalarValue::Int8(_), arrow::datatypes::DataType::Int8) => scalar.try_into(), + (scalar::ScalarValue::Int16(_), arrow::datatypes::DataType::Int16) => scalar.try_into(), + (scalar::ScalarValue::Int32(_), arrow::datatypes::DataType::Int32) => scalar.try_into(), + (scalar::ScalarValue::Int64(_), arrow::datatypes::DataType::Int64) => scalar.try_into(), + (scalar::ScalarValue::UInt8(_), arrow::datatypes::DataType::UInt8) => scalar.try_into(), + (scalar::ScalarValue::UInt16(_), arrow::datatypes::DataType::UInt16) => scalar.try_into(), + (scalar::ScalarValue::UInt32(_), arrow::datatypes::DataType::UInt32) => scalar.try_into(), + (scalar::ScalarValue::UInt64(_), arrow::datatypes::DataType::UInt64) => scalar.try_into(), + (scalar::ScalarValue::Utf8(_), arrow::datatypes::DataType::Utf8) => scalar.try_into(), + (scalar::ScalarValue::LargeUtf8(_), arrow::datatypes::DataType::LargeUtf8) => scalar.try_into(), + _ => Err(proto_error(format!( + "Protobuf serialization error, {:?} was inconsistent with designated type {:?}", + scalar, datatype + ))), + }) + .collect::, _>>()?; + protobuf::ScalarValue { + value: Some(protobuf::scalar_value::Value::ListValue( + protobuf::ScalarListValue { + datatype: Some(datatype.try_into()?), + values: type_checked_values, + }, + )), + } + } + } + None => protobuf::ScalarValue { + value: Some(protobuf::scalar_value::Value::NullListValue( + datatype.try_into()?, + )), + }, + } + } + datafusion::scalar::ScalarValue::Date32(val) => { + create_proto_scalar(val, PrimitiveScalarType::Date32, |s| Value::Date32Value(*s)) + } + datafusion::scalar::ScalarValue::TimeMicrosecond(val) => { + create_proto_scalar(val, PrimitiveScalarType::TimeMicrosecond, |s| { + Value::TimeMicrosecondValue(*s) + }) + } + datafusion::scalar::ScalarValue::TimeNanosecond(val) => { + create_proto_scalar(val, PrimitiveScalarType::TimeNanosecond, |s| { + Value::TimeNanosecondValue(*s) + }) + } + _ => { + return Err(proto_error(format!( + "Error converting to Datatype to scalar type, {:?} is invalid as a datafusion scalar.", + val + ))) + } + }; + Ok(scalar_val) + } +} + +impl TryInto for &LogicalPlan { + type Error = BallistaError; + + fn try_into(self) -> Result { + use protobuf::logical_plan_node::LogicalPlanType; + match self { + LogicalPlan::TableScan { + table_name, + source, + filters, + projection, + .. + } => { + let schema = source.schema(); + + // unwrap the DFTableAdapter to get to the real TableProvider + let source = if let Some(adapter) = source.as_any().downcast_ref::() + { + match &adapter.logical_plan { + LogicalPlan::TableScan { source, .. } => Ok(source.as_any()), + _ => Err(BallistaError::General( + "Invalid LogicalPlan::TableScan".to_owned(), + )), + } + } else { + Ok(source.as_any()) + }?; + + let projection = match projection { + None => None, + Some(columns) => { + let column_names = columns + .iter() + .map(|i| schema.field(*i).name().to_owned()) + .collect(); + Some(protobuf::ProjectionColumns { + columns: column_names, + }) + } + }; + let schema: protobuf::Schema = schema.as_ref().into(); + + let filters: Vec = filters + .iter() + .map(|filter| filter.try_into()) + .collect::, _>>()?; + + if let Some(parquet) = source.downcast_ref::() { + Ok(protobuf::LogicalPlanNode { + logical_plan_type: Some(LogicalPlanType::ParquetScan( + protobuf::ParquetTableScanNode { + table_name: table_name.to_owned(), + path: parquet.path().to_owned(), + projection, + schema: Some(schema), + filters, + }, + )), + }) + } else if let Some(csv) = source.downcast_ref::() { + let delimiter = [csv.delimiter()]; + let delimiter = std::str::from_utf8(&delimiter) + .map_err(|_| BallistaError::General("Invalid CSV delimiter".to_owned()))?; + Ok(protobuf::LogicalPlanNode { + logical_plan_type: Some(LogicalPlanType::CsvScan( + protobuf::CsvTableScanNode { + table_name: table_name.to_owned(), + path: csv.path().to_owned(), + projection, + schema: Some(schema), + has_header: csv.has_header(), + delimiter: delimiter.to_string(), + file_extension: csv.file_extension().to_string(), + filters, + }, + )), + }) + } else { + Err(BallistaError::General(format!( + "logical plan to_proto unsupported table provider {:?}", + source + ))) + } + } + LogicalPlan::Projection { expr, input, .. } => Ok(protobuf::LogicalPlanNode { + logical_plan_type: Some(LogicalPlanType::Projection(Box::new( + protobuf::ProjectionNode { + input: Some(Box::new(input.as_ref().try_into()?)), + expr: expr.iter().map(|expr| expr.try_into()).collect::, + BallistaError, + >>( + )?, + }, + ))), + }), + LogicalPlan::Filter { predicate, input } => { + let input: protobuf::LogicalPlanNode = input.as_ref().try_into()?; + Ok(protobuf::LogicalPlanNode { + logical_plan_type: Some(LogicalPlanType::Selection(Box::new( + protobuf::SelectionNode { + input: Some(Box::new(input)), + expr: Some(predicate.try_into()?), + }, + ))), + }) + } + LogicalPlan::Aggregate { + input, + group_expr, + aggr_expr, + .. + } => { + let input: protobuf::LogicalPlanNode = input.as_ref().try_into()?; + Ok(protobuf::LogicalPlanNode { + logical_plan_type: Some(LogicalPlanType::Aggregate(Box::new( + protobuf::AggregateNode { + input: Some(Box::new(input)), + group_expr: group_expr + .iter() + .map(|expr| expr.try_into()) + .collect::, BallistaError>>()?, + aggr_expr: aggr_expr + .iter() + .map(|expr| expr.try_into()) + .collect::, BallistaError>>()?, + }, + ))), + }) + } + LogicalPlan::Join { + left, + right, + on, + join_type, + .. + } => { + let left: protobuf::LogicalPlanNode = left.as_ref().try_into()?; + let right: protobuf::LogicalPlanNode = right.as_ref().try_into()?; + let join_type = match join_type { + JoinType::Inner => protobuf::JoinType::Inner, + JoinType::Left => protobuf::JoinType::Left, + JoinType::Right => protobuf::JoinType::Right, + }; + let left_join_column = on.iter().map(|on| on.0.to_owned()).collect(); + let right_join_column = on.iter().map(|on| on.1.to_owned()).collect(); + Ok(protobuf::LogicalPlanNode { + logical_plan_type: Some(LogicalPlanType::Join(Box::new(protobuf::JoinNode { + left: Some(Box::new(left)), + right: Some(Box::new(right)), + join_type: join_type.into(), + left_join_column, + right_join_column, + }))), + }) + } + LogicalPlan::Limit { input, n } => { + let input: protobuf::LogicalPlanNode = input.as_ref().try_into()?; + Ok(protobuf::LogicalPlanNode { + logical_plan_type: Some(LogicalPlanType::Limit(Box::new( + protobuf::LimitNode { + input: Some(Box::new(input)), + limit: *n as u32, + }, + ))), + }) + } + LogicalPlan::Sort { input, expr } => { + let input: protobuf::LogicalPlanNode = input.as_ref().try_into()?; + let selection_expr: Vec = expr + .iter() + .map(|expr| expr.try_into()) + .collect::, BallistaError>>()?; + Ok(protobuf::LogicalPlanNode { + logical_plan_type: Some(LogicalPlanType::Sort(Box::new(protobuf::SortNode { + input: Some(Box::new(input)), + expr: selection_expr, + }))), + }) + } + LogicalPlan::Repartition { + input, + partitioning_scheme, + } => { + use datafusion::logical_plan::Partitioning; + let input: protobuf::LogicalPlanNode = input.as_ref().try_into()?; + + //Assumed common usize field was batch size + //Used u64 to avoid any nastyness involving large values, most data clusters are probably uniformly 64 bits any ways + use protobuf::repartition_node::PartitionMethod; + + let pb_partition_method = match partitioning_scheme { + Partitioning::Hash(exprs, partition_count) => { + PartitionMethod::Hash(protobuf::HashRepartition { + hash_expr: exprs.iter().map(|expr| expr.try_into()).collect::, + BallistaError, + >>( + )?, + partition_count: *partition_count as u64, + }) + } + Partitioning::RoundRobinBatch(batch_size) => { + PartitionMethod::RoundRobin(*batch_size as u64) + } + }; + + Ok(protobuf::LogicalPlanNode { + logical_plan_type: Some(LogicalPlanType::Repartition(Box::new( + protobuf::RepartitionNode { + input: Some(Box::new(input)), + partition_method: Some(pb_partition_method), + }, + ))), + }) + } + LogicalPlan::EmptyRelation { + produce_one_row, .. + } => Ok(protobuf::LogicalPlanNode { + logical_plan_type: Some(LogicalPlanType::EmptyRelation( + protobuf::EmptyRelationNode { + produce_one_row: *produce_one_row, + }, + )), + }), + LogicalPlan::CreateExternalTable { + name, + location, + file_type, + has_header, + schema: df_schema, + } => { + use datafusion::sql::parser::FileType; + let schema: Schema = df_schema.as_ref().clone().into(); + let pb_schema: protobuf::Schema = (&schema).try_into().map_err(|e| { + BallistaError::General(format!( + "Could not convert schema into protobuf: {:?}", + e + )) + })?; + + let pb_file_type: protobuf::FileType = match file_type { + FileType::NdJson => protobuf::FileType::NdJson, + FileType::Parquet => protobuf::FileType::Parquet, + FileType::CSV => protobuf::FileType::Csv, + }; + + Ok(protobuf::LogicalPlanNode { + logical_plan_type: Some(LogicalPlanType::CreateExternalTable( + protobuf::CreateExternalTableNode { + name: name.clone(), + location: location.clone(), + file_type: pb_file_type as i32, + has_header: *has_header, + schema: Some(pb_schema), + }, + )), + }) + } + LogicalPlan::Explain { verbose, plan, .. } => { + let input: protobuf::LogicalPlanNode = plan.as_ref().try_into()?; + Ok(protobuf::LogicalPlanNode { + logical_plan_type: Some(LogicalPlanType::Explain(Box::new( + protobuf::ExplainNode { + input: Some(Box::new(input)), + verbose: *verbose, + }, + ))), + }) + } + LogicalPlan::Extension { .. } => unimplemented!(), + // _ => Err(BallistaError::General(format!( + // "logical plan to_proto {:?}", + // self + // ))), + } + } +} + +fn create_proto_scalar protobuf::scalar_value::Value>( + v: &Option, + null_arrow_type: protobuf::PrimitiveScalarType, + constructor: T, +) -> protobuf::ScalarValue { + protobuf::ScalarValue { + value: Some(v.as_ref().map(constructor).unwrap_or( + protobuf::scalar_value::Value::NullValue(null_arrow_type as i32), + )), + } +} + +impl TryInto for &Expr { + type Error = BallistaError; + + fn try_into(self) -> Result { + use datafusion::scalar::ScalarValue; + use protobuf::scalar_value::Value; + match self { + Expr::Column(name) => { + let expr = protobuf::LogicalExprNode { + expr_type: Some(ExprType::ColumnName(name.clone())), + }; + Ok(expr) + } + Expr::Alias(expr, alias) => { + let alias = Box::new(protobuf::AliasNode { + expr: Some(Box::new(expr.as_ref().try_into()?)), + alias: alias.to_owned(), + }); + let expr = protobuf::LogicalExprNode { + expr_type: Some(ExprType::Alias(alias)), + }; + Ok(expr) + } + Expr::Literal(value) => { + let pb_value: protobuf::ScalarValue = value.try_into()?; + Ok(protobuf::LogicalExprNode { + expr_type: Some(ExprType::Literal(pb_value)), + }) + } + Expr::BinaryExpr { left, op, right } => { + let binary_expr = Box::new(protobuf::BinaryExprNode { + l: Some(Box::new(left.as_ref().try_into()?)), + r: Some(Box::new(right.as_ref().try_into()?)), + op: format!("{:?}", op), + }); + Ok(protobuf::LogicalExprNode { + expr_type: Some(ExprType::BinaryExpr(binary_expr)), + }) + } + Expr::AggregateFunction { + ref fun, ref args, .. + } => { + let aggr_function = match fun { + AggregateFunction::Min => protobuf::AggregateFunction::Min, + AggregateFunction::Max => protobuf::AggregateFunction::Max, + AggregateFunction::Sum => protobuf::AggregateFunction::Sum, + AggregateFunction::Avg => protobuf::AggregateFunction::Avg, + AggregateFunction::Count => protobuf::AggregateFunction::Count, + }; + + let arg = &args[0]; + let aggregate_expr = Box::new(protobuf::AggregateExprNode { + aggr_function: aggr_function.into(), + expr: Some(Box::new(arg.try_into()?)), + }); + Ok(protobuf::LogicalExprNode { + expr_type: Some(ExprType::AggregateExpr(aggregate_expr)), + }) + } + Expr::ScalarVariable(_) => unimplemented!(), + Expr::ScalarFunction { ref fun, ref args } => { + let fun: protobuf::ScalarFunction = fun.try_into()?; + let expr: Vec = + args.iter() + .map(|e| Ok(e.try_into()?)) + .collect::, BallistaError>>()?; + Ok(protobuf::LogicalExprNode { + expr_type: Some(protobuf::logical_expr_node::ExprType::ScalarFunction( + protobuf::ScalarFunctionNode { + fun: fun.into(), + expr, + }, + )), + }) + } + Expr::ScalarUDF { .. } => unimplemented!(), + Expr::AggregateUDF { .. } => unimplemented!(), + Expr::Not(expr) => { + let expr = Box::new(protobuf::Not { + expr: Some(Box::new(expr.as_ref().try_into()?)), + }); + Ok(protobuf::LogicalExprNode { + expr_type: Some(ExprType::NotExpr(expr)), + }) + } + Expr::IsNull(expr) => { + let expr = Box::new(protobuf::IsNull { + expr: Some(Box::new(expr.as_ref().try_into()?)), + }); + Ok(protobuf::LogicalExprNode { + expr_type: Some(ExprType::IsNullExpr(expr)), + }) + } + Expr::IsNotNull(expr) => { + let expr = Box::new(protobuf::IsNotNull { + expr: Some(Box::new(expr.as_ref().try_into()?)), + }); + Ok(protobuf::LogicalExprNode { + expr_type: Some(ExprType::IsNotNullExpr(expr)), + }) + } + Expr::Between { + expr, + negated, + low, + high, + } => { + let expr = Box::new(protobuf::BetweenNode { + expr: Some(Box::new(expr.as_ref().try_into()?)), + negated: *negated, + low: Some(Box::new(low.as_ref().try_into()?)), + high: Some(Box::new(high.as_ref().try_into()?)), + }); + Ok(protobuf::LogicalExprNode { + expr_type: Some(ExprType::Between(expr)), + }) + } + Expr::Case { + expr, + when_then_expr, + else_expr, + } => { + let when_then_expr = when_then_expr + .iter() + .map(|(w, t)| { + Ok(protobuf::WhenThen { + when_expr: Some(w.as_ref().try_into()?), + then_expr: Some(t.as_ref().try_into()?), + }) + }) + .collect::, BallistaError>>()?; + let expr = Box::new(protobuf::CaseNode { + expr: match expr { + Some(e) => Some(Box::new(e.as_ref().try_into()?)), + None => None, + }, + when_then_expr, + else_expr: match else_expr { + Some(e) => Some(Box::new(e.as_ref().try_into()?)), + None => None, + }, + }); + Ok(protobuf::LogicalExprNode { + expr_type: Some(ExprType::Case(expr)), + }) + } + Expr::Cast { expr, data_type } => { + let expr = Box::new(protobuf::CastNode { + expr: Some(Box::new(expr.as_ref().try_into()?)), + arrow_type: Some(data_type.into()), + }); + Ok(protobuf::LogicalExprNode { + expr_type: Some(ExprType::Cast(expr)), + }) + } + Expr::Sort { + expr, + asc, + nulls_first, + } => { + let expr = Box::new(protobuf::SortExprNode { + expr: Some(Box::new(expr.as_ref().try_into()?)), + asc: *asc, + nulls_first: *nulls_first, + }); + Ok(protobuf::LogicalExprNode { + expr_type: Some(ExprType::Sort(expr)), + }) + } + Expr::Negative(expr) => { + let expr = Box::new(protobuf::NegativeNode { + expr: Some(Box::new(expr.as_ref().try_into()?)), + }); + Ok(protobuf::LogicalExprNode { + expr_type: Some(protobuf::logical_expr_node::ExprType::Negative(expr)), + }) + } + Expr::InList { + expr, + list, + negated, + } => { + let expr = Box::new(protobuf::InListNode { + expr: Some(Box::new(expr.as_ref().try_into()?)), + list: list + .iter() + .map(|expr| expr.try_into()) + .collect::, BallistaError>>()?, + negated: *negated, + }); + Ok(protobuf::LogicalExprNode { + expr_type: Some(protobuf::logical_expr_node::ExprType::InList(expr)), + }) + } + Expr::Wildcard => Ok(protobuf::LogicalExprNode { + expr_type: Some(protobuf::logical_expr_node::ExprType::Wildcard(true)), + }), + // _ => Err(BallistaError::General(format!( + // "logical expr to_proto {:?}", + // self + // ))), + } + } +} + +impl Into for &Schema { + fn into(self) -> protobuf::Schema { + protobuf::Schema { + columns: self + .fields() + .iter() + .map(protobuf::Field::from) + .collect::>(), + } + } +} + +impl TryFrom<&arrow::datatypes::DataType> for protobuf::ScalarType { + type Error = BallistaError; + fn try_from(value: &arrow::datatypes::DataType) -> Result { + let datatype = protobuf::scalar_type::Datatype::try_from(value)?; + Ok(protobuf::ScalarType { + datatype: Some(datatype), + }) + } +} + +impl TryInto for &BuiltinScalarFunction { + type Error = BallistaError; + fn try_into(self) -> Result { + match self { + BuiltinScalarFunction::Sqrt => Ok(protobuf::ScalarFunction::Sqrt), + BuiltinScalarFunction::Sin => Ok(protobuf::ScalarFunction::Sin), + BuiltinScalarFunction::Cos => Ok(protobuf::ScalarFunction::Cos), + BuiltinScalarFunction::Tan => Ok(protobuf::ScalarFunction::Tan), + BuiltinScalarFunction::Asin => Ok(protobuf::ScalarFunction::Asin), + BuiltinScalarFunction::Acos => Ok(protobuf::ScalarFunction::Acos), + BuiltinScalarFunction::Atan => Ok(protobuf::ScalarFunction::Atan), + BuiltinScalarFunction::Exp => Ok(protobuf::ScalarFunction::Exp), + BuiltinScalarFunction::Log => Ok(protobuf::ScalarFunction::Log), + BuiltinScalarFunction::Log10 => Ok(protobuf::ScalarFunction::Log10), + BuiltinScalarFunction::Floor => Ok(protobuf::ScalarFunction::Floor), + BuiltinScalarFunction::Ceil => Ok(protobuf::ScalarFunction::Ceil), + BuiltinScalarFunction::Round => Ok(protobuf::ScalarFunction::Round), + BuiltinScalarFunction::Trunc => Ok(protobuf::ScalarFunction::Trunc), + BuiltinScalarFunction::Abs => Ok(protobuf::ScalarFunction::Abs), + BuiltinScalarFunction::OctetLength => Ok(protobuf::ScalarFunction::Octetlength), + BuiltinScalarFunction::Concat => Ok(protobuf::ScalarFunction::Concat), + BuiltinScalarFunction::Lower => Ok(protobuf::ScalarFunction::Lower), + BuiltinScalarFunction::Upper => Ok(protobuf::ScalarFunction::Upper), + BuiltinScalarFunction::Trim => Ok(protobuf::ScalarFunction::Trim), + BuiltinScalarFunction::Ltrim => Ok(protobuf::ScalarFunction::Ltrim), + BuiltinScalarFunction::Rtrim => Ok(protobuf::ScalarFunction::Rtrim), + BuiltinScalarFunction::ToTimestamp => Ok(protobuf::ScalarFunction::Totimestamp), + BuiltinScalarFunction::Array => Ok(protobuf::ScalarFunction::Array), + BuiltinScalarFunction::NullIf => Ok(protobuf::ScalarFunction::Nullif), + BuiltinScalarFunction::DateTrunc => Ok(protobuf::ScalarFunction::Datetrunc), + BuiltinScalarFunction::MD5 => Ok(protobuf::ScalarFunction::Md5), + BuiltinScalarFunction::SHA224 => Ok(protobuf::ScalarFunction::Sha224), + BuiltinScalarFunction::SHA256 => Ok(protobuf::ScalarFunction::Sha256), + BuiltinScalarFunction::SHA384 => Ok(protobuf::ScalarFunction::Sha384), + BuiltinScalarFunction::SHA512 => Ok(protobuf::ScalarFunction::Sha512), + _ => Err(BallistaError::General(format!( + "logical_plan::to_proto() unsupported scalar function {:?}", + self + ))), + } + } +} diff --git a/rust/ballista/rust/core/src/serde/mod.rs b/rust/ballista/rust/core/src/serde/mod.rs new file mode 100644 index 0000000000000..b96163999f39d --- /dev/null +++ b/rust/ballista/rust/core/src/serde/mod.rs @@ -0,0 +1,69 @@ +// 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. + +//! This crate contains code generated from the Ballista Protocol Buffer Definition as well +//! as convenience code for interacting with the generated code. + +use std::{convert::TryInto, io::Cursor}; + +use crate::{error::BallistaError, serde::scheduler::Action as BallistaAction}; + +use prost::Message; + +// include the generated protobuf source as a submodule +#[allow(clippy::all)] +pub mod protobuf { + include!(concat!(env!("OUT_DIR"), "/ballista.protobuf.rs")); +} + +pub mod logical_plan; +pub mod physical_plan; +pub mod scheduler; + +pub fn decode_protobuf(bytes: &[u8]) -> Result { + let mut buf = Cursor::new(bytes); + + protobuf::Action::decode(&mut buf) + .map_err(|e| BallistaError::Internal(format!("{:?}", e))) + .and_then(|node| node.try_into()) +} + +pub(crate) fn proto_error>(message: S) -> BallistaError { + BallistaError::General(message.into()) +} + +#[macro_export] +macro_rules! convert_required { + ($PB:expr) => {{ + if let Some(field) = $PB.as_ref() { + field.try_into() + } else { + Err(proto_error("Missing required field in protobuf")) + } + }}; +} + +#[macro_export] +macro_rules! convert_box_required { + ($PB:expr) => {{ + if let Some(field) = $PB.as_ref() { + field.as_ref().try_into() + } else { + Err(proto_error("Missing required field in protobuf")) + } + }}; +} diff --git a/rust/ballista/rust/core/src/serde/physical_plan/from_proto.rs b/rust/ballista/rust/core/src/serde/physical_plan/from_proto.rs new file mode 100644 index 0000000000000..2b82706db9230 --- /dev/null +++ b/rust/ballista/rust/core/src/serde/physical_plan/from_proto.rs @@ -0,0 +1,367 @@ +// 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. + +//! Serde code to convert from protocol buffers to Rust data structures. + +use std::collections::HashMap; +use std::convert::TryInto; +use std::sync::Arc; + +use crate::error::BallistaError; +use crate::execution_plans::{ShuffleReaderExec, UnresolvedShuffleExec}; +use crate::serde::protobuf::repartition_exec_node::PartitionMethod; +use crate::serde::protobuf::LogicalExprNode; +use crate::serde::scheduler::PartitionLocation; +use crate::serde::{proto_error, protobuf}; +use crate::{convert_box_required, convert_required}; + +use arrow::datatypes::{DataType, Schema, SchemaRef}; +use datafusion::execution::context::{ExecutionConfig, ExecutionContextState}; +use datafusion::logical_plan::{DFSchema, Expr}; +use datafusion::physical_plan::aggregates::{create_aggregate_expr, AggregateFunction}; +use datafusion::physical_plan::expressions::col; +use datafusion::physical_plan::hash_aggregate::{AggregateMode, HashAggregateExec}; +use datafusion::physical_plan::merge::MergeExec; +use datafusion::physical_plan::planner::DefaultPhysicalPlanner; +use datafusion::physical_plan::{ + coalesce_batches::CoalesceBatchesExec, + csv::CsvExec, + empty::EmptyExec, + expressions::{Avg, Column, PhysicalSortExpr}, + filter::FilterExec, + hash_join::HashJoinExec, + hash_utils::JoinType, + limit::{GlobalLimitExec, LocalLimitExec}, + parquet::ParquetExec, + projection::ProjectionExec, + repartition::RepartitionExec, + sort::{SortExec, SortOptions}, + Partitioning, +}; +use datafusion::physical_plan::{AggregateExpr, ExecutionPlan, PhysicalExpr}; +use datafusion::prelude::CsvReadOptions; +use log::debug; +use protobuf::logical_expr_node::ExprType; +use protobuf::physical_plan_node::PhysicalPlanType; + +impl TryInto> for &protobuf::PhysicalPlanNode { + type Error = BallistaError; + + fn try_into(self) -> Result, Self::Error> { + let plan = self.physical_plan_type.as_ref().ok_or_else(|| { + proto_error(format!( + "physical_plan::from_proto() Unsupported physical plan '{:?}'", + self + )) + })?; + match plan { + PhysicalPlanType::Projection(projection) => { + let input: Arc = convert_box_required!(projection.input)?; + let exprs = projection + .expr + .iter() + .zip(projection.expr_name.iter()) + .map(|(expr, name)| { + compile_expr(expr, &input.schema()).map(|e| (e, name.to_string())) + }) + .collect::, _>>()?; + Ok(Arc::new(ProjectionExec::try_new(exprs, input)?)) + } + PhysicalPlanType::Filter(filter) => { + let input: Arc = convert_box_required!(filter.input)?; + let predicate = compile_expr( + filter.expr.as_ref().ok_or_else(|| { + BallistaError::General( + "filter (FilterExecNode) in PhysicalPlanNode is missing.".to_owned(), + ) + })?, + &input.schema(), + )?; + Ok(Arc::new(FilterExec::try_new(predicate, input)?)) + } + PhysicalPlanType::CsvScan(scan) => { + let schema = Arc::new(convert_required!(scan.schema)?); + let options = CsvReadOptions::new() + .has_header(scan.has_header) + .file_extension(&scan.file_extension) + .delimiter(scan.delimiter.as_bytes()[0]) + .schema(&schema); + // TODO we don't care what the DataFusion batch size was because Ballista will + // have its own configs. Hard-code for now. + let batch_size = 32768; + let projection = scan.projection.iter().map(|i| *i as usize).collect(); + Ok(Arc::new(CsvExec::try_new( + &scan.path, + options, + Some(projection), + batch_size, + )?)) + } + PhysicalPlanType::ParquetScan(scan) => { + let projection = scan.projection.iter().map(|i| *i as usize).collect(); + let filenames: Vec<&str> = scan.filename.iter().map(|s| s.as_str()).collect(); + Ok(Arc::new(ParquetExec::try_from_files( + &filenames, + Some(projection), + None, + scan.batch_size as usize, + scan.num_partitions as usize, + )?)) + } + PhysicalPlanType::CoalesceBatches(coalesce_batches) => { + let input: Arc = convert_box_required!(coalesce_batches.input)?; + Ok(Arc::new(CoalesceBatchesExec::new( + input, + coalesce_batches.target_batch_size as usize, + ))) + } + PhysicalPlanType::Merge(merge) => { + let input: Arc = convert_box_required!(merge.input)?; + Ok(Arc::new(MergeExec::new(input))) + } + PhysicalPlanType::Repartition(repart) => { + let input: Arc = convert_box_required!(repart.input)?; + match repart.partition_method { + Some(PartitionMethod::Hash(ref hash_part)) => { + let expr = hash_part + .hash_expr + .iter() + .map(|e| compile_expr(e, &input.schema())) + .collect::>, _>>()?; + + Ok(Arc::new(RepartitionExec::try_new( + input, + Partitioning::Hash(expr, hash_part.partition_count.try_into().unwrap()), + )?)) + } + Some(PartitionMethod::RoundRobin(partition_count)) => { + Ok(Arc::new(RepartitionExec::try_new( + input, + Partitioning::RoundRobinBatch(partition_count.try_into().unwrap()), + )?)) + } + Some(PartitionMethod::Unknown(partition_count)) => { + Ok(Arc::new(RepartitionExec::try_new( + input, + Partitioning::UnknownPartitioning(partition_count.try_into().unwrap()), + )?)) + } + _ => Err(BallistaError::General( + "Invalid partitioning scheme".to_owned(), + )), + } + } + PhysicalPlanType::GlobalLimit(limit) => { + let input: Arc = convert_box_required!(limit.input)?; + Ok(Arc::new(GlobalLimitExec::new(input, limit.limit as usize))) + } + PhysicalPlanType::LocalLimit(limit) => { + let input: Arc = convert_box_required!(limit.input)?; + Ok(Arc::new(LocalLimitExec::new(input, limit.limit as usize))) + } + PhysicalPlanType::HashAggregate(hash_agg) => { + let input: Arc = convert_box_required!(hash_agg.input)?; + let mode = protobuf::AggregateMode::from_i32(hash_agg.mode).ok_or_else(|| { + proto_error(format!( + "Received a HashAggregateNode message with unknown AggregateMode {}", + hash_agg.mode + )) + })?; + let agg_mode: AggregateMode = match mode { + protobuf::AggregateMode::Partial => AggregateMode::Partial, + protobuf::AggregateMode::Final => AggregateMode::Final, + }; + + let group = hash_agg + .group_expr + .iter() + .zip(hash_agg.group_expr_name.iter()) + .map(|(expr, name)| { + compile_expr(expr, &input.schema()).map(|e| (e, name.to_string())) + }) + .collect::, _>>()?; + + let logical_agg_expr: Vec<(Expr, String)> = hash_agg + .aggr_expr + .iter() + .zip(hash_agg.aggr_expr_name.iter()) + .map(|(expr, name)| expr.try_into().map(|expr| (expr, name.clone()))) + .collect::, _>>()?; + + let df_planner = DefaultPhysicalPlanner::default(); + let ctx_state = ExecutionContextState { + datasources: Default::default(), + scalar_functions: Default::default(), + var_provider: Default::default(), + aggregate_functions: Default::default(), + config: ExecutionConfig::new(), + }; + + let input_schema = hash_agg + .input_schema + .as_ref() + .ok_or_else(|| { + BallistaError::General( + "input_schema in HashAggregateNode is missing.".to_owned(), + ) + })? + .clone(); + let physical_schema: SchemaRef = SchemaRef::new((&input_schema).try_into()?); + + let mut physical_aggr_expr = vec![]; + + for (expr, name) in &logical_agg_expr { + match expr { + Expr::AggregateFunction { fun, args, .. } => { + let arg = df_planner + .create_physical_expr(&args[0], &physical_schema, &ctx_state) + .map_err(|e| BallistaError::General(format!("{:?}", e)))?; + physical_aggr_expr.push(create_aggregate_expr( + &fun, + false, + &[arg], + &physical_schema, + name.to_string(), + )?); + } + _ => { + return Err(BallistaError::General( + "Invalid expression for HashAggregateExec".to_string(), + )) + } + } + } + Ok(Arc::new(HashAggregateExec::try_new( + agg_mode, + group, + physical_aggr_expr, + input, + Arc::new((&input_schema).try_into()?), + )?)) + } + PhysicalPlanType::HashJoin(hashjoin) => { + let left: Arc = convert_box_required!(hashjoin.left)?; + let right: Arc = convert_box_required!(hashjoin.right)?; + let on: Vec<(String, String)> = hashjoin + .on + .iter() + .map(|col| (col.left.clone(), col.right.clone())) + .collect(); + let join_type = + protobuf::JoinType::from_i32(hashjoin.join_type).ok_or_else(|| { + proto_error(format!( + "Received a HashJoinNode message with unknown JoinType {}", + hashjoin.join_type + )) + })?; + let join_type = match join_type { + protobuf::JoinType::Inner => JoinType::Inner, + protobuf::JoinType::Left => JoinType::Left, + protobuf::JoinType::Right => JoinType::Right, + }; + Ok(Arc::new(HashJoinExec::try_new( + left, right, &on, &join_type, + )?)) + } + PhysicalPlanType::ShuffleReader(shuffle_reader) => { + let schema = Arc::new(convert_required!(shuffle_reader.schema)?); + let partition_location: Vec = shuffle_reader + .partition_location + .iter() + .map(|p| p.clone().try_into()) + .collect::, BallistaError>>()?; + let shuffle_reader = ShuffleReaderExec::try_new(partition_location, schema)?; + Ok(Arc::new(shuffle_reader)) + } + PhysicalPlanType::Empty(empty) => { + let schema = Arc::new(convert_required!(empty.schema)?); + Ok(Arc::new(EmptyExec::new(empty.produce_one_row, schema))) + } + PhysicalPlanType::Sort(sort) => { + let input: Arc = convert_box_required!(sort.input)?; + let exprs = sort + .expr + .iter() + .map(|expr| { + let expr = expr.expr_type.as_ref().ok_or_else(|| { + proto_error(format!( + "physical_plan::from_proto() Unexpected expr {:?}", + self + )) + })?; + if let protobuf::logical_expr_node::ExprType::Sort(sort_expr) = expr { + let expr = sort_expr + .expr + .as_ref() + .ok_or_else(|| { + proto_error(format!( + "physical_plan::from_proto() Unexpected sort expr {:?}", + self + )) + })? + .as_ref(); + Ok(PhysicalSortExpr { + expr: compile_expr(expr, &input.schema())?, + options: SortOptions { + descending: !sort_expr.asc, + nulls_first: sort_expr.nulls_first, + }, + }) + } else { + Err(BallistaError::General(format!( + "physical_plan::from_proto() {:?}", + self + ))) + } + }) + .collect::, _>>()?; + // Update concurrency here in the future + Ok(Arc::new(SortExec::try_new(exprs, input)?)) + } + PhysicalPlanType::Unresolved(unresolved_shuffle) => { + let schema = Arc::new(convert_required!(unresolved_shuffle.schema)?); + Ok(Arc::new(UnresolvedShuffleExec { + query_stage_ids: unresolved_shuffle + .query_stage_ids + .iter() + .map(|id| *id as usize) + .collect(), + schema, + partition_count: unresolved_shuffle.partition_count as usize, + })) + } + } + } +} + +fn compile_expr( + expr: &protobuf::LogicalExprNode, + schema: &Schema, +) -> Result, BallistaError> { + let df_planner = DefaultPhysicalPlanner::default(); + let state = ExecutionContextState { + datasources: HashMap::new(), + scalar_functions: HashMap::new(), + var_provider: HashMap::new(), + aggregate_functions: HashMap::new(), + config: ExecutionConfig::new(), + }; + let expr: Expr = expr.try_into()?; + df_planner + .create_physical_expr(&expr, schema, &state) + .map_err(|e| BallistaError::General(format!("{:?}", e))) +} diff --git a/rust/ballista/rust/core/src/serde/physical_plan/mod.rs b/rust/ballista/rust/core/src/serde/physical_plan/mod.rs new file mode 100644 index 0000000000000..4d8a189ab7cda --- /dev/null +++ b/rust/ballista/rust/core/src/serde/physical_plan/mod.rs @@ -0,0 +1,173 @@ +// 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. + +pub mod from_proto; +pub mod to_proto; + +#[cfg(test)] +mod roundtrip_tests { + use datafusion::physical_plan::hash_utils::JoinType; + use std::{convert::TryInto, sync::Arc}; + + use arrow::datatypes::{DataType, Schema}; + use datafusion::physical_plan::ColumnarValue; + use datafusion::physical_plan::{ + empty::EmptyExec, + expressions::{Avg, Column, PhysicalSortExpr}, + hash_aggregate::{AggregateMode, HashAggregateExec}, + hash_join::HashJoinExec, + limit::{GlobalLimitExec, LocalLimitExec}, + sort::SortExec, + ExecutionPlan, + }; + use datafusion::physical_plan::{AggregateExpr, Distribution, Partitioning, PhysicalExpr}; + + use super::super::super::error::Result; + use super::super::protobuf; + + fn roundtrip_test(exec_plan: Arc) -> Result<()> { + let proto: protobuf::PhysicalPlanNode = exec_plan.clone().try_into()?; + let result_exec_plan: Arc = (&proto).try_into()?; + assert_eq!( + format!("{:?}", exec_plan), + format!("{:?}", result_exec_plan) + ); + Ok(()) + } + + #[test] + fn roundtrip_empty() -> Result<()> { + roundtrip_test(Arc::new(EmptyExec::new(false, Arc::new(Schema::empty())))) + } + + #[test] + fn roundtrip_local_limit() -> Result<()> { + roundtrip_test(Arc::new(LocalLimitExec::new( + Arc::new(EmptyExec::new(false, Arc::new(Schema::empty()))), + 25, + ))) + } + + #[test] + fn roundtrip_global_limit() -> Result<()> { + roundtrip_test(Arc::new(GlobalLimitExec::new( + Arc::new(EmptyExec::new(false, Arc::new(Schema::empty()))), + 25, + ))) + } + + #[test] + fn roundtrip_hash_join() -> Result<()> { + use arrow::datatypes::{DataType, Field, Schema}; + let field_a = Field::new("col", DataType::Int64, false); + let schema_left = Schema::new(vec![field_a.clone()]); + let schema_right = Schema::new(vec![field_a]); + + roundtrip_test(Arc::new(HashJoinExec::try_new( + Arc::new(EmptyExec::new(false, Arc::new(schema_left))), + Arc::new(EmptyExec::new(false, Arc::new(schema_right))), + &[("col".to_string(), "col".to_string())], + &JoinType::Inner, + )?)) + } + + fn col(name: &str) -> Arc { + Arc::new(Column::new(name)) + } + + #[test] + fn rountrip_hash_aggregate() -> Result<()> { + use arrow::datatypes::{DataType, Field, Schema}; + let groups: Vec<(Arc, String)> = vec![(col("a"), "unused".to_string())]; + + let aggregates: Vec> = vec![Arc::new(Avg::new( + col("b"), + "AVG(b)".to_string(), + DataType::Float64, + ))]; + + let field_a = Field::new("a", DataType::Int64, false); + let field_b = Field::new("b", DataType::Int64, false); + let schema = Arc::new(Schema::new(vec![field_a, field_b])); + + roundtrip_test(Arc::new(HashAggregateExec::try_new( + AggregateMode::Final, + groups.clone(), + aggregates.clone(), + Arc::new(EmptyExec::new(false, schema.clone())), + schema, + )?)) + } + + #[test] + fn roundtrip_filter_with_not_and_in_list() -> Result<()> { + use arrow::datatypes::{DataType, Field, Schema}; + use datafusion::logical_plan::Operator; + use datafusion::physical_plan::{ + expressions::{binary, lit, InListExpr, NotExpr}, + filter::FilterExec, + }; + use datafusion::scalar::ScalarValue; + let field_a = Field::new("a", DataType::Boolean, false); + let field_b = Field::new("b", DataType::Int64, false); + let field_c = Field::new("c", DataType::Int64, false); + let schema = Arc::new(Schema::new(vec![field_a, field_b, field_c])); + let not = Arc::new(NotExpr::new(col("a"))); + let in_list = Arc::new(InListExpr::new( + col("b"), + vec![ + lit(ScalarValue::Int64(Some(1))), + lit(ScalarValue::Int64(Some(2))), + ], + false, + )); + let and = binary(not, Operator::And, in_list, &schema)?; + roundtrip_test(Arc::new(FilterExec::try_new( + and, + Arc::new(EmptyExec::new(false, schema.clone())), + )?)) + } + + #[test] + fn roundtrip_sort() -> Result<()> { + use arrow::compute::kernels::sort::SortOptions; + use arrow::datatypes::{DataType, Field, Schema}; + let field_a = Field::new("a", DataType::Boolean, false); + let field_b = Field::new("b", DataType::Int64, false); + let schema = Arc::new(Schema::new(vec![field_a, field_b])); + let sort_exprs = vec![ + PhysicalSortExpr { + expr: col("a"), + options: SortOptions { + descending: true, + nulls_first: false, + }, + }, + PhysicalSortExpr { + expr: col("b"), + options: SortOptions { + descending: false, + nulls_first: true, + }, + }, + ]; + roundtrip_test(Arc::new(SortExec::try_new( + sort_exprs, + Arc::new(EmptyExec::new(false, schema)), + )?)) + } +} diff --git a/rust/ballista/rust/core/src/serde/physical_plan/to_proto.rs b/rust/ballista/rust/core/src/serde/physical_plan/to_proto.rs new file mode 100644 index 0000000000000..3aab3d2bc8b53 --- /dev/null +++ b/rust/ballista/rust/core/src/serde/physical_plan/to_proto.rs @@ -0,0 +1,525 @@ +// 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.language governing permissions and +// limitations under the License. + +//! Serde code to convert Arrow schemas and DataFusion logical plans to Ballista protocol +//! buffer format, allowing DataFusion physical plans to be serialized and transmitted between +//! processes. + +use std::{ + convert::{TryFrom, TryInto}, + str::FromStr, + sync::Arc, +}; + +use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec; +use datafusion::physical_plan::csv::CsvExec; +use datafusion::physical_plan::expressions::CastExpr; +use datafusion::physical_plan::expressions::{ + CaseExpr, InListExpr, IsNotNullExpr, IsNullExpr, NegativeExpr, NotExpr, +}; +use datafusion::physical_plan::filter::FilterExec; +use datafusion::physical_plan::hash_aggregate::AggregateMode; +use datafusion::physical_plan::hash_join::HashJoinExec; +use datafusion::physical_plan::hash_utils::JoinType; +use datafusion::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; +use datafusion::physical_plan::parquet::ParquetExec; +use datafusion::physical_plan::projection::ProjectionExec; +use datafusion::physical_plan::sort::SortExec; +use datafusion::{ + physical_plan::expressions::{Count, Literal}, + scalar::ScalarValue, +}; + +use datafusion::physical_plan::{ + empty::EmptyExec, + expressions::{Avg, BinaryExpr, Column, Sum}, + Partitioning, +}; +use datafusion::physical_plan::{AggregateExpr, ExecutionPlan, PhysicalExpr}; + +use datafusion::physical_plan::hash_aggregate::HashAggregateExec; +use protobuf::physical_plan_node::PhysicalPlanType; + +use crate::execution_plans::{ShuffleReaderExec, UnresolvedShuffleExec}; +use crate::serde::protobuf::repartition_exec_node::PartitionMethod; +use crate::serde::{protobuf, BallistaError}; +use datafusion::physical_plan::functions::{BuiltinScalarFunction, ScalarFunctionExpr}; +use datafusion::physical_plan::merge::MergeExec; +use datafusion::physical_plan::repartition::RepartitionExec; + +impl TryInto for Arc { + type Error = BallistaError; + + fn try_into(self) -> Result { + let plan = self.as_any(); + + if let Some(exec) = plan.downcast_ref::() { + let input: protobuf::PhysicalPlanNode = exec.input().to_owned().try_into()?; + let expr = exec + .expr() + .iter() + .map(|expr| expr.0.clone().try_into()) + .collect::, Self::Error>>()?; + let expr_name = exec.expr().iter().map(|expr| expr.1.clone()).collect(); + Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::Projection(Box::new( + protobuf::ProjectionExecNode { + input: Some(Box::new(input)), + expr, + expr_name, + }, + ))), + }) + } else if let Some(exec) = plan.downcast_ref::() { + let input: protobuf::PhysicalPlanNode = exec.input().to_owned().try_into()?; + Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::Filter(Box::new( + protobuf::FilterExecNode { + input: Some(Box::new(input)), + expr: Some(exec.predicate().clone().try_into()?), + }, + ))), + }) + } else if let Some(limit) = plan.downcast_ref::() { + let input: protobuf::PhysicalPlanNode = limit.input().to_owned().try_into()?; + Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::GlobalLimit(Box::new( + protobuf::GlobalLimitExecNode { + input: Some(Box::new(input)), + limit: limit.limit() as u32, + }, + ))), + }) + } else if let Some(limit) = plan.downcast_ref::() { + let input: protobuf::PhysicalPlanNode = limit.input().to_owned().try_into()?; + Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::LocalLimit(Box::new( + protobuf::LocalLimitExecNode { + input: Some(Box::new(input)), + limit: limit.limit() as u32, + }, + ))), + }) + } else if let Some(exec) = plan.downcast_ref::() { + let left: protobuf::PhysicalPlanNode = exec.left().to_owned().try_into()?; + let right: protobuf::PhysicalPlanNode = exec.right().to_owned().try_into()?; + let on: Vec = exec + .on() + .iter() + .map(|tuple| protobuf::JoinOn { + left: tuple.0.to_owned(), + right: tuple.1.to_owned(), + }) + .collect(); + let join_type = match exec.join_type() { + JoinType::Inner => protobuf::JoinType::Inner, + JoinType::Left => protobuf::JoinType::Left, + JoinType::Right => protobuf::JoinType::Right, + }; + Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::HashJoin(Box::new( + protobuf::HashJoinExecNode { + left: Some(Box::new(left)), + right: Some(Box::new(right)), + on, + join_type: join_type.into(), + }, + ))), + }) + } else if let Some(exec) = plan.downcast_ref::() { + let groups = exec + .group_expr() + .iter() + .map(|expr| expr.0.to_owned().try_into()) + .collect::, BallistaError>>()?; + let group_names = exec + .group_expr() + .iter() + .map(|expr| expr.1.to_owned()) + .collect(); + let agg = exec + .aggr_expr() + .iter() + .map(|expr| expr.to_owned().try_into()) + .collect::, BallistaError>>()?; + let agg_names = exec + .aggr_expr() + .iter() + .map(|expr| match expr.field() { + Ok(field) => Ok(field.name().clone()), + Err(e) => Err(BallistaError::DataFusionError(e)), + }) + .collect::>()?; + + let agg_mode = match exec.mode() { + AggregateMode::Partial => protobuf::AggregateMode::Partial, + AggregateMode::Final => protobuf::AggregateMode::Final, + }; + let input_schema = exec.input_schema(); + let input: protobuf::PhysicalPlanNode = exec.input().to_owned().try_into()?; + Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::HashAggregate(Box::new( + protobuf::HashAggregateExecNode { + group_expr: groups, + group_expr_name: group_names, + aggr_expr: agg, + aggr_expr_name: agg_names, + mode: agg_mode as i32, + input: Some(Box::new(input)), + input_schema: Some(input_schema.as_ref().into()), + }, + ))), + }) + } else if let Some(empty) = plan.downcast_ref::() { + let schema = empty.schema().as_ref().into(); + Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::Empty(protobuf::EmptyExecNode { + produce_one_row: empty.produce_one_row(), + schema: Some(schema), + })), + }) + } else if let Some(coalesce_batches) = plan.downcast_ref::() { + let input: protobuf::PhysicalPlanNode = + coalesce_batches.input().to_owned().try_into()?; + Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::CoalesceBatches(Box::new( + protobuf::CoalesceBatchesExecNode { + input: Some(Box::new(input)), + target_batch_size: coalesce_batches.target_batch_size() as u32, + }, + ))), + }) + } else if let Some(exec) = plan.downcast_ref::() { + let delimiter = [*exec.delimiter().ok_or_else(|| { + BallistaError::General("Delimeter is not set for CsvExec".to_owned()) + })?]; + let delimiter = std::str::from_utf8(&delimiter) + .map_err(|_| BallistaError::General("Invalid CSV delimiter".to_owned()))?; + + Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::CsvScan(protobuf::CsvScanExecNode { + path: exec.path().to_owned(), + filename: exec.filenames().to_vec(), + projection: exec + .projection() + .ok_or_else(|| { + BallistaError::General( + "projection in CsvExec dosn not exist.".to_owned(), + ) + })? + .iter() + .map(|n| *n as u32) + .collect(), + file_extension: exec.file_extension().to_owned(), + schema: Some(exec.file_schema().as_ref().into()), + has_header: exec.has_header(), + delimiter: delimiter.to_string(), + batch_size: 32768, + })), + }) + } else if let Some(exec) = plan.downcast_ref::() { + let filenames = exec + .partitions() + .iter() + .flat_map(|part| part.filenames().to_owned()) + .collect(); + Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::ParquetScan( + protobuf::ParquetScanExecNode { + filename: filenames, + projection: exec + .projection() + .as_ref() + .iter() + .map(|n| *n as u32) + .collect(), + num_partitions: exec.partitions().len() as u32, + batch_size: exec.batch_size() as u32, + }, + )), + }) + } else if let Some(exec) = plan.downcast_ref::() { + let partition_location = exec + .partition_location + .iter() + .map(|l| l.clone().try_into()) + .collect::>()?; + + Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::ShuffleReader( + protobuf::ShuffleReaderExecNode { + partition_location, + schema: Some(exec.schema().as_ref().into()), + }, + )), + }) + } else if let Some(exec) = plan.downcast_ref::() { + let input: protobuf::PhysicalPlanNode = exec.input().to_owned().try_into()?; + Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::Merge(Box::new( + protobuf::MergeExecNode { + input: Some(Box::new(input)), + }, + ))), + }) + } else if let Some(exec) = plan.downcast_ref::() { + let input: protobuf::PhysicalPlanNode = exec.input().to_owned().try_into()?; + + let pb_partition_method = match exec.partitioning() { + Partitioning::Hash(exprs, partition_count) => { + PartitionMethod::Hash(protobuf::HashRepartition { + hash_expr: exprs + .iter() + .map(|expr| expr.clone().try_into()) + .collect::, BallistaError>>()?, + partition_count: *partition_count as u64, + }) + } + Partitioning::RoundRobinBatch(partition_count) => { + PartitionMethod::RoundRobin(*partition_count as u64) + } + Partitioning::UnknownPartitioning(partition_count) => { + PartitionMethod::Unknown(*partition_count as u64) + } + }; + + Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::Repartition(Box::new( + protobuf::RepartitionExecNode { + input: Some(Box::new(input)), + partition_method: Some(pb_partition_method), + }, + ))), + }) + } else if let Some(exec) = plan.downcast_ref::() { + let input: protobuf::PhysicalPlanNode = exec.input().to_owned().try_into()?; + let expr = exec + .expr() + .iter() + .map(|expr| { + let sort_expr = Box::new(protobuf::SortExprNode { + expr: Some(Box::new(expr.expr.to_owned().try_into()?)), + asc: !expr.options.descending, + nulls_first: expr.options.nulls_first, + }); + Ok(protobuf::LogicalExprNode { + expr_type: Some(protobuf::logical_expr_node::ExprType::Sort(sort_expr)), + }) + }) + .collect::, Self::Error>>()?; + Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::Sort(Box::new( + protobuf::SortExecNode { + input: Some(Box::new(input)), + expr, + }, + ))), + }) + } else if let Some(exec) = plan.downcast_ref::() { + Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::Unresolved( + protobuf::UnresolvedShuffleExecNode { + query_stage_ids: exec.query_stage_ids.iter().map(|id| *id as u32).collect(), + schema: Some(exec.schema().as_ref().into()), + partition_count: exec.partition_count as u32, + }, + )), + }) + } else { + Err(BallistaError::General(format!( + "physical plan to_proto unsupported plan {:?}", + self + ))) + } + } +} + +impl TryInto for Arc { + type Error = BallistaError; + + fn try_into(self) -> Result { + let aggr_function = if self.as_any().downcast_ref::().is_some() { + Ok(protobuf::AggregateFunction::Avg.into()) + } else if self.as_any().downcast_ref::().is_some() { + Ok(protobuf::AggregateFunction::Sum.into()) + } else if self.as_any().downcast_ref::().is_some() { + Ok(protobuf::AggregateFunction::Count.into()) + } else { + Err(BallistaError::NotImplemented(format!( + "Aggregate function not supported: {:?}", + self + ))) + }?; + let expressions: Vec = self + .expressions() + .iter() + .map(|e| e.clone().try_into()) + .collect::, BallistaError>>()?; + Ok(protobuf::LogicalExprNode { + expr_type: Some(protobuf::logical_expr_node::ExprType::AggregateExpr( + Box::new(protobuf::AggregateExprNode { + aggr_function, + expr: Some(Box::new(expressions[0].clone())), + }), + )), + }) + } +} + +impl TryFrom> for protobuf::LogicalExprNode { + type Error = BallistaError; + + fn try_from(value: Arc) -> Result { + let expr = value.as_any(); + + if let Some(expr) = expr.downcast_ref::() { + Ok(protobuf::LogicalExprNode { + expr_type: Some(protobuf::logical_expr_node::ExprType::ColumnName( + expr.name().to_owned(), + )), + }) + } else if let Some(expr) = expr.downcast_ref::() { + let binary_expr = Box::new(protobuf::BinaryExprNode { + l: Some(Box::new(expr.left().to_owned().try_into()?)), + r: Some(Box::new(expr.right().to_owned().try_into()?)), + op: format!("{:?}", expr.op()), + }); + + Ok(protobuf::LogicalExprNode { + expr_type: Some(protobuf::logical_expr_node::ExprType::BinaryExpr( + binary_expr, + )), + }) + } else if let Some(expr) = expr.downcast_ref::() { + Ok(protobuf::LogicalExprNode { + expr_type: Some(protobuf::logical_expr_node::ExprType::Case(Box::new( + protobuf::CaseNode { + expr: expr + .expr() + .as_ref() + .map(|exp| exp.clone().try_into().map(Box::new)) + .transpose()?, + when_then_expr: expr + .when_then_expr() + .iter() + .map(|(when_expr, then_expr)| { + try_parse_when_then_expr(when_expr, then_expr) + }) + .collect::, Self::Error>>()?, + else_expr: expr + .else_expr() + .map(|a| a.clone().try_into().map(Box::new)) + .transpose()?, + }, + ))), + }) + } else if let Some(expr) = expr.downcast_ref::() { + Ok(protobuf::LogicalExprNode { + expr_type: Some(protobuf::logical_expr_node::ExprType::NotExpr(Box::new( + protobuf::Not { + expr: Some(Box::new(expr.arg().to_owned().try_into()?)), + }, + ))), + }) + } else if let Some(expr) = expr.downcast_ref::() { + Ok(protobuf::LogicalExprNode { + expr_type: Some(protobuf::logical_expr_node::ExprType::IsNullExpr(Box::new( + protobuf::IsNull { + expr: Some(Box::new(expr.arg().to_owned().try_into()?)), + }, + ))), + }) + } else if let Some(expr) = expr.downcast_ref::() { + Ok(protobuf::LogicalExprNode { + expr_type: Some(protobuf::logical_expr_node::ExprType::IsNotNullExpr( + Box::new(protobuf::IsNotNull { + expr: Some(Box::new(expr.arg().to_owned().try_into()?)), + }), + )), + }) + } else if let Some(expr) = expr.downcast_ref::() { + Ok(protobuf::LogicalExprNode { + expr_type: Some(protobuf::logical_expr_node::ExprType::InList(Box::new( + protobuf::InListNode { + expr: Some(Box::new(expr.expr().to_owned().try_into()?)), + list: expr + .list() + .iter() + .map(|a| a.clone().try_into()) + .collect::, Self::Error>>()?, + negated: expr.negated(), + }, + ))), + }) + } else if let Some(expr) = expr.downcast_ref::() { + Ok(protobuf::LogicalExprNode { + expr_type: Some(protobuf::logical_expr_node::ExprType::Negative(Box::new( + protobuf::NegativeNode { + expr: Some(Box::new(expr.arg().to_owned().try_into()?)), + }, + ))), + }) + } else if let Some(lit) = expr.downcast_ref::() { + Ok(protobuf::LogicalExprNode { + expr_type: Some(protobuf::logical_expr_node::ExprType::Literal( + lit.value().try_into()?, + )), + }) + } else if let Some(cast) = expr.downcast_ref::() { + Ok(protobuf::LogicalExprNode { + expr_type: Some(protobuf::logical_expr_node::ExprType::Cast(Box::new( + protobuf::CastNode { + expr: Some(Box::new(cast.expr().clone().try_into()?)), + arrow_type: Some(cast.cast_type().into()), + }, + ))), + }) + } else if let Some(expr) = expr.downcast_ref::() { + let fun: BuiltinScalarFunction = BuiltinScalarFunction::from_str(expr.name())?; + let fun: protobuf::ScalarFunction = (&fun).try_into()?; + let expr: Vec = expr + .args() + .iter() + .map(|e| e.to_owned().try_into()) + .collect::, _>>()?; + Ok(protobuf::LogicalExprNode { + expr_type: Some(protobuf::logical_expr_node::ExprType::ScalarFunction( + protobuf::ScalarFunctionNode { + fun: fun.into(), + expr, + }, + )), + }) + } else { + Err(BallistaError::General(format!( + "physical_plan::to_proto() unsupported expression {:?}", + value + ))) + } + } +} + +fn try_parse_when_then_expr( + when_expr: &Arc, + then_expr: &Arc, +) -> Result { + Ok(protobuf::WhenThen { + when_expr: Some(when_expr.clone().try_into()?), + then_expr: Some(then_expr.clone().try_into()?), + }) +} diff --git a/rust/ballista/rust/core/src/serde/scheduler/from_proto.rs b/rust/ballista/rust/core/src/serde/scheduler/from_proto.rs new file mode 100644 index 0000000000000..0ba7abd2bd546 --- /dev/null +++ b/rust/ballista/rust/core/src/serde/scheduler/from_proto.rs @@ -0,0 +1,123 @@ +// 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. + +use std::{collections::HashMap, convert::TryInto}; + +use crate::error::BallistaError; +use crate::serde::protobuf; +use crate::serde::protobuf::action::ActionType; +use crate::serde::scheduler::{ + Action, ExecutePartition, PartitionId, PartitionLocation, PartitionStats, +}; + +use datafusion::logical_plan::LogicalPlan; +use uuid::Uuid; + +impl TryInto for protobuf::Action { + type Error = BallistaError; + + fn try_into(self) -> Result { + match self.action_type { + Some(ActionType::ExecutePartition(partition)) => { + Ok(Action::ExecutePartition(ExecutePartition::new( + partition.job_id, + partition.stage_id as usize, + partition.partition_id.iter().map(|n| *n as usize).collect(), + partition + .plan + .as_ref() + .ok_or_else(|| { + BallistaError::General( + "PhysicalPlanNode in ExecutePartition is missing".to_owned(), + ) + })? + .try_into()?, + HashMap::new(), + ))) + } + Some(ActionType::FetchPartition(partition)) => { + Ok(Action::FetchPartition(partition.try_into()?)) + } + _ => Err(BallistaError::General( + "scheduler::from_proto(Action) invalid or missing action".to_owned(), + )), + } + } +} + +impl TryInto for protobuf::PartitionId { + type Error = BallistaError; + + fn try_into(self) -> Result { + Ok(PartitionId::new( + &self.job_id, + self.stage_id as usize, + self.partition_id as usize, + )) + } +} + +impl Into for protobuf::PartitionStats { + fn into(self) -> PartitionStats { + PartitionStats::new( + foo(self.num_rows), + foo(self.num_batches), + foo(self.num_bytes), + ) + } +} + +fn foo(n: i64) -> Option { + if n < 0 { + None + } else { + Some(n as u64) + } +} + +impl TryInto for protobuf::PartitionLocation { + type Error = BallistaError; + + fn try_into(self) -> Result { + Ok(PartitionLocation { + partition_id: self + .partition_id + .ok_or_else(|| { + BallistaError::General( + "partition_id in PartitionLocation is missing.".to_owned(), + ) + })? + .try_into()?, + executor_meta: self + .executor_meta + .ok_or_else(|| { + BallistaError::General( + "executor_meta in PartitionLocation is missing".to_owned(), + ) + })? + .into(), + partition_stats: self + .partition_stats + .ok_or_else(|| { + BallistaError::General( + "partition_stats in PartitionLocation is missing".to_owned(), + ) + })? + .into(), + }) + } +} diff --git a/rust/ballista/rust/core/src/serde/scheduler/mod.rs b/rust/ballista/rust/core/src/serde/scheduler/mod.rs new file mode 100644 index 0000000000000..40c200f151132 --- /dev/null +++ b/rust/ballista/rust/core/src/serde/scheduler/mod.rs @@ -0,0 +1,256 @@ +// 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. + +use std::{collections::HashMap, sync::Arc}; + +use arrow::array::{ + ArrayBuilder, ArrayRef, StructArray, StructBuilder, UInt64Array, UInt64Builder, +}; +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use datafusion::logical_plan::LogicalPlan; +use datafusion::physical_plan::ExecutionPlan; +use uuid::Uuid; + +use super::protobuf; +use crate::error::BallistaError; + +pub mod from_proto; +pub mod to_proto; + +/// Action that can be sent to an executor +#[derive(Debug, Clone)] + +pub enum Action { + /// Execute a query and store the results in memory + ExecutePartition(ExecutePartition), + /// Collect a shuffle partition + FetchPartition(PartitionId), +} + +/// Unique identifier for the output partition of an operator. +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Hash)] +pub struct PartitionId { + pub job_id: String, + pub stage_id: usize, + pub partition_id: usize, +} + +impl PartitionId { + pub fn new(job_id: &str, stage_id: usize, partition_id: usize) -> Self { + Self { + job_id: job_id.to_string(), + stage_id, + partition_id, + } + } +} + +#[derive(Debug, Clone)] +pub struct PartitionLocation { + pub partition_id: PartitionId, + pub executor_meta: ExecutorMeta, + pub partition_stats: PartitionStats, +} + +/// Meta-data for an executor, used when fetching shuffle partitions from other executors +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct ExecutorMeta { + pub id: String, + pub host: String, + pub port: u16, +} + +impl Into for ExecutorMeta { + fn into(self) -> protobuf::ExecutorMetadata { + protobuf::ExecutorMetadata { + id: self.id, + host: self.host, + port: self.port as u32, + } + } +} + +impl From for ExecutorMeta { + fn from(meta: protobuf::ExecutorMetadata) -> Self { + Self { + id: meta.id, + host: meta.host, + port: meta.port as u16, + } + } +} + +/// Summary of executed partition +#[derive(Debug, Copy, Clone)] +pub struct PartitionStats { + num_rows: Option, + num_batches: Option, + num_bytes: Option, +} + +impl Default for PartitionStats { + fn default() -> Self { + Self { + num_rows: None, + num_batches: None, + num_bytes: None, + } + } +} + +impl PartitionStats { + pub fn new(num_rows: Option, num_batches: Option, num_bytes: Option) -> Self { + Self { + num_rows, + num_batches, + num_bytes, + } + } + + pub fn arrow_struct_repr(self) -> Field { + Field::new( + "partition_stats", + DataType::Struct(self.arrow_struct_fields()), + false, + ) + } + fn arrow_struct_fields(self) -> Vec { + vec![ + Field::new("num_rows", DataType::UInt64, false), + Field::new("num_batches", DataType::UInt64, false), + Field::new("num_bytes", DataType::UInt64, false), + ] + } + + pub fn to_arrow_arrayref(&self) -> Result, BallistaError> { + let mut field_builders = Vec::new(); + + let mut num_rows_builder = UInt64Builder::new(1); + match self.num_rows { + Some(n) => num_rows_builder.append_value(n)?, + None => num_rows_builder.append_null()?, + } + field_builders.push(Box::new(num_rows_builder) as Box); + + let mut num_batches_builder = UInt64Builder::new(1); + match self.num_batches { + Some(n) => num_batches_builder.append_value(n)?, + None => num_batches_builder.append_null()?, + } + field_builders.push(Box::new(num_batches_builder) as Box); + + let mut num_bytes_builder = UInt64Builder::new(1); + match self.num_bytes { + Some(n) => num_bytes_builder.append_value(n)?, + None => num_bytes_builder.append_null()?, + } + field_builders.push(Box::new(num_bytes_builder) as Box); + + let mut struct_builder = StructBuilder::new(self.arrow_struct_fields(), field_builders); + struct_builder.append(true)?; + Ok(Arc::new(struct_builder.finish())) + } + + pub fn from_arrow_struct_array(struct_array: &StructArray) -> PartitionStats { + let num_rows = struct_array + .column_by_name("num_rows") + .expect("from_arrow_struct_array expected a field num_rows") + .as_any() + .downcast_ref::() + .expect("from_arrow_struct_array expected num_rows to be a UInt64Array"); + let num_batches = struct_array + .column_by_name("num_batches") + .expect("from_arrow_struct_array expected a field num_batches") + .as_any() + .downcast_ref::() + .expect("from_arrow_struct_array expected num_batches to be a UInt64Array"); + let num_bytes = struct_array + .column_by_name("num_bytes") + .expect("from_arrow_struct_array expected a field num_bytes") + .as_any() + .downcast_ref::() + .expect("from_arrow_struct_array expected num_bytes to be a UInt64Array"); + PartitionStats { + num_rows: Some(num_rows.value(0).to_owned()), + num_batches: Some(num_batches.value(0).to_owned()), + num_bytes: Some(num_bytes.value(0).to_owned()), + } + } +} + +/// Task that can be sent to an executor to execute one stage of a query and write +/// results out to disk +#[derive(Debug, Clone)] +pub struct ExecutePartition { + /// Unique ID representing this query execution + pub job_id: String, + /// Unique ID representing this query stage within the overall query + pub stage_id: usize, + /// The partitions to execute. The same plan could be sent to multiple executors and each + /// executor will execute a range of partitions per QueryStageTask + pub partition_id: Vec, + /// The physical plan for this query stage + pub plan: Arc, + /// Location of shuffle partitions that this query stage may depend on + pub shuffle_locations: HashMap, +} + +impl ExecutePartition { + pub fn new( + job_id: String, + stage_id: usize, + partition_id: Vec, + plan: Arc, + shuffle_locations: HashMap, + ) -> Self { + Self { + job_id, + stage_id, + partition_id, + plan, + shuffle_locations, + } + } + + pub fn key(&self) -> String { + format!("{}.{}.{:?}", self.job_id, self.stage_id, self.partition_id) + } +} + +#[derive(Debug)] +pub struct ExecutePartitionResult { + /// Path containing results for this partition + path: String, + stats: PartitionStats, +} + +impl ExecutePartitionResult { + pub fn new(path: &str, stats: PartitionStats) -> Self { + Self { + path: path.to_owned(), + stats, + } + } + + pub fn path(&self) -> &str { + &self.path + } + + pub fn statistics(&self) -> &PartitionStats { + &self.stats + } +} diff --git a/rust/ballista/rust/core/src/serde/scheduler/to_proto.rs b/rust/ballista/rust/core/src/serde/scheduler/to_proto.rs new file mode 100644 index 0000000000000..f581becdea176 --- /dev/null +++ b/rust/ballista/rust/core/src/serde/scheduler/to_proto.rs @@ -0,0 +1,90 @@ +// 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. + +use std::convert::TryInto; + +use crate::error::BallistaError; +use crate::serde::protobuf; +use crate::serde::protobuf::action::ActionType; +use crate::serde::scheduler::{ + Action, ExecutePartition, PartitionId, PartitionLocation, PartitionStats, +}; + +impl TryInto for Action { + type Error = BallistaError; + + fn try_into(self) -> Result { + match self { + Action::ExecutePartition(partition) => Ok(protobuf::Action { + action_type: Some(ActionType::ExecutePartition(partition.try_into()?)), + settings: vec![], + }), + Action::FetchPartition(partition_id) => Ok(protobuf::Action { + action_type: Some(ActionType::FetchPartition(partition_id.into())), + settings: vec![], + }), + } + } +} + +impl TryInto for ExecutePartition { + type Error = BallistaError; + + fn try_into(self) -> Result { + Ok(protobuf::ExecutePartition { + job_id: self.job_id, + stage_id: self.stage_id as u32, + partition_id: self.partition_id.iter().map(|n| *n as u32).collect(), + plan: Some(self.plan.try_into()?), + partition_location: vec![], + }) + } +} + +impl Into for PartitionId { + fn into(self) -> protobuf::PartitionId { + protobuf::PartitionId { + job_id: self.job_id, + stage_id: self.stage_id as u32, + partition_id: self.partition_id as u32, + } + } +} + +impl TryInto for PartitionLocation { + type Error = BallistaError; + + fn try_into(self) -> Result { + Ok(protobuf::PartitionLocation { + partition_id: Some(self.partition_id.into()), + executor_meta: Some(self.executor_meta.into()), + partition_stats: Some(self.partition_stats.into()), + }) + } +} + +impl Into for PartitionStats { + fn into(self) -> protobuf::PartitionStats { + let none_value = -1_i64; + protobuf::PartitionStats { + num_rows: self.num_rows.map(|n| n as i64).unwrap_or(none_value), + num_batches: self.num_batches.map(|n| n as i64).unwrap_or(none_value), + num_bytes: self.num_bytes.map(|n| n as i64).unwrap_or(none_value), + column_stats: vec![], + } + } +} diff --git a/rust/ballista/rust/core/src/utils.rs b/rust/ballista/rust/core/src/utils.rs new file mode 100644 index 0000000000000..9814901bd2116 --- /dev/null +++ b/rust/ballista/rust/core/src/utils.rs @@ -0,0 +1,293 @@ +// 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. + +use std::collections::HashMap; +use std::io::{BufWriter, Write}; +use std::ops::Deref; +use std::sync::atomic::{AtomicUsize, Ordering}; +use std::sync::Arc; +use std::{fs::File, pin::Pin}; + +use crate::error::{BallistaError, Result}; +use crate::execution_plans::{QueryStageExec, UnresolvedShuffleExec}; +use crate::memory_stream::MemoryStream; +use crate::serde::scheduler::PartitionStats; +use arrow::array::{ + ArrayBuilder, ArrayRef, StructArray, StructBuilder, UInt64Array, UInt64Builder, +}; +use arrow::datatypes::{DataType, Field}; +use arrow::ipc::reader::FileReader; +use arrow::ipc::writer::FileWriter; +use arrow::record_batch::RecordBatch; +use datafusion::logical_plan::Operator; +use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec; +use datafusion::physical_plan::csv::CsvExec; +use datafusion::physical_plan::expressions::{BinaryExpr, Column, Literal}; +use datafusion::physical_plan::filter::FilterExec; +use datafusion::physical_plan::hash_aggregate::HashAggregateExec; +use datafusion::physical_plan::hash_join::HashJoinExec; +use datafusion::physical_plan::merge::MergeExec; +use datafusion::physical_plan::parquet::ParquetExec; +use datafusion::physical_plan::projection::ProjectionExec; +use datafusion::physical_plan::sort::SortExec; +use datafusion::physical_plan::{AggregateExpr, ExecutionPlan, PhysicalExpr, RecordBatchStream}; +use futures::StreamExt; + +/// Stream data to disk in Arrow IPC format + +pub async fn write_stream_to_disk( + stream: &mut Pin>, + path: &str, +) -> Result { + let file = File::create(&path).map_err(|e| { + BallistaError::General(format!( + "Failed to create partition file at {}: {:?}", + path, e + )) + })?; + + let mut num_rows = 0; + let mut num_batches = 0; + let mut num_bytes = 0; + let mut writer = FileWriter::try_new(file, stream.schema().as_ref())?; + + while let Some(result) = stream.next().await { + let batch = result?; + + let batch_size_bytes: usize = batch + .columns() + .iter() + .map(|array| array.get_array_memory_size()) + .sum(); + num_batches += 1; + num_rows += batch.num_rows(); + num_bytes += batch_size_bytes; + writer.write(&batch)?; + } + writer.finish()?; + Ok(PartitionStats::new( + Some(num_rows as u64), + Some(num_batches), + Some(num_bytes as u64), + )) +} + +pub async fn collect_stream( + stream: &mut Pin>, +) -> Result> { + let mut batches = vec![]; + while let Some(batch) = stream.next().await { + batches.push(batch?); + } + Ok(batches) +} + +pub fn format_plan(plan: &dyn ExecutionPlan, indent: usize) -> Result { + let operator_str = if let Some(exec) = plan.as_any().downcast_ref::() { + format!( + "HashAggregateExec: groupBy={:?}, aggrExpr={:?}", + exec.group_expr() + .iter() + .map(|e| format_expr(e.0.as_ref())) + .collect::>(), + exec.aggr_expr() + .iter() + .map(|e| format_agg_expr(e.as_ref())) + .collect::>>()? + ) + } else if let Some(exec) = plan.as_any().downcast_ref::() { + format!( + "HashJoinExec: joinType={:?}, on={:?}", + exec.join_type(), + exec.on() + ) + } else if let Some(exec) = plan.as_any().downcast_ref::() { + let mut num_files = 0; + for part in exec.partitions() { + num_files += part.filenames().len(); + } + format!( + "ParquetExec: partitions={}, files={}", + exec.partitions().len(), + num_files + ) + } else if let Some(exec) = plan.as_any().downcast_ref::() { + format!( + "CsvExec: {}; partitions={}", + &exec.path(), + exec.output_partitioning().partition_count() + ) + } else if let Some(exec) = plan.as_any().downcast_ref::() { + format!("FilterExec: {}", format_expr(exec.predicate().as_ref())) + } else if let Some(exec) = plan.as_any().downcast_ref::() { + format!( + "QueryStageExec: job={}, stage={}", + exec.job_id, exec.stage_id + ) + } else if let Some(exec) = plan.as_any().downcast_ref::() { + format!("UnresolvedShuffleExec: stages={:?}", exec.query_stage_ids) + } else if let Some(exec) = plan.as_any().downcast_ref::() { + format!( + "CoalesceBatchesExec: batchSize={}", + exec.target_batch_size() + ) + } else if plan.as_any().downcast_ref::().is_some() { + "MergeExec".to_string() + } else { + let str = format!("{:?}", plan); + String::from(&str[0..120]) + }; + + let children_str = plan + .children() + .iter() + .map(|c| format_plan(c.as_ref(), indent + 1)) + .collect::>>()? + .join("\n"); + + let indent_str = " ".repeat(indent); + if plan.children().is_empty() { + Ok(format!("{}{}{}", indent_str, &operator_str, children_str)) + } else { + Ok(format!("{}{}\n{}", indent_str, &operator_str, children_str)) + } +} + +pub fn format_agg_expr(expr: &dyn AggregateExpr) -> Result { + Ok(format!( + "{} {:?}", + expr.field()?.name(), + expr.expressions() + .iter() + .map(|e| format_expr(e.as_ref())) + .collect::>() + )) +} + +pub fn format_expr(expr: &dyn PhysicalExpr) -> String { + if let Some(e) = expr.as_any().downcast_ref::() { + e.name().to_string() + } else if let Some(e) = expr.as_any().downcast_ref::() { + e.to_string() + } else if let Some(e) = expr.as_any().downcast_ref::() { + format!("{} {} {}", e.left(), e.op(), e.right()) + } else { + format!("{}", expr) + } +} + +pub fn produce_diagram(filename: &str, stages: &[Arc]) -> Result<()> { + let write_file = File::create(filename)?; + let mut w = BufWriter::new(&write_file); + writeln!(w, "digraph G {{")?; + + // draw stages and entities + for stage in stages { + writeln!(w, "\tsubgraph cluster{} {{", stage.stage_id)?; + writeln!(w, "\t\tlabel = \"Stage {}\";", stage.stage_id)?; + let mut id = AtomicUsize::new(0); + build_exec_plan_diagram(&mut w, stage.child.as_ref(), stage.stage_id, &mut id, true)?; + writeln!(w, "\t}}")?; + } + + // draw relationships + for stage in stages { + let mut id = AtomicUsize::new(0); + build_exec_plan_diagram(&mut w, stage.child.as_ref(), stage.stage_id, &mut id, false)?; + } + + write!(w, "}}")?; + Ok(()) +} + +fn build_exec_plan_diagram( + w: &mut BufWriter<&File>, + plan: &dyn ExecutionPlan, + stage_id: usize, + id: &mut AtomicUsize, + draw_entity: bool, +) -> Result { + let operator_str = if plan.as_any().downcast_ref::().is_some() { + "HashAggregateExec" + } else if plan.as_any().downcast_ref::().is_some() { + "SortExec" + } else if plan.as_any().downcast_ref::().is_some() { + "ProjectionExec" + } else if plan.as_any().downcast_ref::().is_some() { + "HashJoinExec" + } else if plan.as_any().downcast_ref::().is_some() { + "ParquetExec" + } else if plan.as_any().downcast_ref::().is_some() { + "CsvExec" + } else if plan.as_any().downcast_ref::().is_some() { + "FilterExec" + } else if plan.as_any().downcast_ref::().is_some() { + "QueryStageExec" + } else if plan + .as_any() + .downcast_ref::() + .is_some() + { + "UnresolvedShuffleExec" + } else if plan + .as_any() + .downcast_ref::() + .is_some() + { + "CoalesceBatchesExec" + } else if plan.as_any().downcast_ref::().is_some() { + "MergeExec" + } else { + println!("Unknown: {:?}", plan); + "Unknown" + }; + + let node_id = id.load(Ordering::SeqCst); + id.store(node_id + 1, Ordering::SeqCst); + + if draw_entity { + writeln!( + w, + "\t\tstage_{}_exec_{} [shape=box, label=\"{}\"];", + stage_id, node_id, operator_str + )?; + } + for child in plan.children() { + if let Some(shuffle) = child.as_any().downcast_ref::() { + if !draw_entity { + for y in &shuffle.query_stage_ids { + writeln!( + w, + "\tstage_{}_exec_1 -> stage_{}_exec_{};", + y, stage_id, node_id + )?; + } + } + } else { + // relationships within same entity + let child_id = build_exec_plan_diagram(w, child.as_ref(), stage_id, id, draw_entity)?; + if draw_entity { + writeln!( + w, + "\t\tstage_{}_exec_{} -> stage_{}_exec_{};", + stage_id, child_id, stage_id, node_id + )?; + } + } + } + Ok(node_id) +} diff --git a/rust/ballista/rust/executor/Cargo.toml b/rust/ballista/rust/executor/Cargo.toml new file mode 100644 index 0000000000000..743b62cc1001d --- /dev/null +++ b/rust/ballista/rust/executor/Cargo.toml @@ -0,0 +1,59 @@ +# 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. + +[package] +name = "ballista-executor" +description = "Ballista Distributed Compute - Executor" +license = "Apache-2.0" +version = "0.4.2-SNAPSHOT" +homepage = "https://github.com/apache/arrow" +repository = "https://github.com/apache/arrow" +authors = ["Apache Arrow "] +edition = "2018" + +[features] +default = ["snmalloc"] +snmalloc = ["snmalloc-rs"] + +[dependencies] +anyhow = "1" +async-trait = "0.1.36" +ballista-core = { path = "../core" } +ballista-scheduler = { path = "../scheduler" } +configure_me = "0.4.0" +env_logger = "0.8" +futures = "0.3" +log = "0.4" +snmalloc-rs = {version = "0.2", features= ["cache-friendly"], optional = true} +tempfile = "3" +tokio = { version = "1.0", features = ["macros", "rt", "rt-multi-thread"] } +tokio-stream = "0.1" +tonic = "0.4" +uuid = { version = "0.8", features = ["v4"] } + +arrow = { git = "https://github.com/apache/arrow", rev="46161d2" } +arrow-flight = { git = "https://github.com/apache/arrow", rev="46161d2" } +datafusion = { git = "https://github.com/apache/arrow", rev="46161d2" } + +[dev-dependencies] + +[build-dependencies] +configure_me_codegen = "0.4.0" + +[package.metadata.configure_me.bin] +executor = "executor_config_spec.toml" + diff --git a/rust/ballista/rust/executor/README.md b/rust/ballista/rust/executor/README.md new file mode 100644 index 0000000000000..c0824e639fdc8 --- /dev/null +++ b/rust/ballista/rust/executor/README.md @@ -0,0 +1,31 @@ + + +# Ballista Executor - Rust +This crate contains the Ballista Executor. It can be used both as a library or as a binary. + +## Run + +```bash +RUST_LOG=info cargo run --release +... +[2021-02-11T05:30:13Z INFO executor] Running with config: ExecutorConfig { host: "localhost", port: 50051, work_dir: "/var/folders/y8/fc61kyjd4n53tn444n72rjrm0000gn/T/.tmpv1LjN0", concurrent_tasks: 4 } +``` + +By default, the executor will bind to `localhost` and listen on port `50051`. \ No newline at end of file diff --git a/rust/ballista/rust/executor/build.rs b/rust/ballista/rust/executor/build.rs new file mode 100644 index 0000000000000..1c9e32b0b8943 --- /dev/null +++ b/rust/ballista/rust/executor/build.rs @@ -0,0 +1,24 @@ +// 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. + +extern crate configure_me_codegen; + +fn main() -> Result<(), String> { + println!("cargo:rerun-if-changed=executor_config_spec.toml"); + configure_me_codegen::build_script_auto() + .map_err(|e| format!("configure_me code generation failed: {}", e)) +} diff --git a/rust/ballista/rust/executor/examples/example_executor_config.toml b/rust/ballista/rust/executor/examples/example_executor_config.toml new file mode 100644 index 0000000000000..0705016ff30f3 --- /dev/null +++ b/rust/ballista/rust/executor/examples/example_executor_config.toml @@ -0,0 +1,22 @@ +# 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. + +# the default configuration location is "/etc/ballista/scheduler.toml" +# if you include a specifc conf file using "--config-file = my_config_file.toml" +# then that file will override environment variables, but not command line arguments +namespace = "my_name_space" +bind_host = "1.2.3.4" \ No newline at end of file diff --git a/rust/ballista/rust/executor/executor_config_spec.toml b/rust/ballista/rust/executor/executor_config_spec.toml new file mode 100644 index 0000000000000..cb47ca064236d --- /dev/null +++ b/rust/ballista/rust/executor/executor_config_spec.toml @@ -0,0 +1,79 @@ +# 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. + +[general] +name = "Ballista Executor" +env_prefix = "BALLISTA_EXECUTOR" +conf_file_param = "config_file" + +[[switch]] +name = "version" +doc = "Print version of this executable" + +[[param]] +abbr = "n" +name = "namespace" +type = "String" +doc = "Namespace for the ballista cluster that this executor will join. yippee" +default = "std::string::String::from(\"ballista\")" + +[[param]] +name = "scheduler_host" +type = "String" +default = "std::string::String::from(\"localhost\")" +doc = "Scheduler host" + +[[param]] +name = "scheduler_port" +type = "u16" +default = "50050" +doc = "scheduler port" + +[[switch]] +name = "local" +doc = "Running in local mode will launch a standalone scheduler inside the executor process. This will create a single-executor cluster, and is useful for development scenarios." + +[[param]] +name = "bind_host" +type = "String" +default = "std::string::String::from(\"0.0.0.0\")" +doc = "Local IP address to bind to." + +[[param]] +name = "external_host" +type = "String" +default = "std::string::String::from(\"localhost\")" +doc = "Host name or IP address to register with scheduler so that other executors can connect to this executor." + +[[param]] +abbr = "p" +name = "port" +type = "u16" +default = "50051" +doc = "bind port" + +[[param]] +name = "work_dir" +type = "String" +doc = "Directory for temporary IPC files" + +[[param]] +abbr = "c" +name = "concurrent_tasks" +type = "usize" +default = "4" +doc = "Max concurrent tasks." \ No newline at end of file diff --git a/rust/ballista/rust/executor/src/collect.rs b/rust/ballista/rust/executor/src/collect.rs new file mode 100644 index 0000000000000..0c65d9394cf6d --- /dev/null +++ b/rust/ballista/rust/executor/src/collect.rs @@ -0,0 +1,124 @@ +// 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. + +//! The CollectExec operator retrieves results from the cluster and returns them as a single +//! vector of [RecordBatch]. + +use std::sync::Arc; +use std::task::{Context, Poll}; +use std::{any::Any, pin::Pin}; + +use arrow::datatypes::SchemaRef; +use arrow::error::Result as ArrowResult; +use arrow::record_batch::RecordBatch; +use async_trait::async_trait; +use datafusion::error::DataFusionError; +use datafusion::physical_plan::{ExecutionPlan, Partitioning, SendableRecordBatchStream}; +use datafusion::{error::Result, physical_plan::RecordBatchStream}; +use futures::stream::SelectAll; +use futures::Stream; + +/// The CollectExec operator retrieves results from the cluster and returns them as a single +/// vector of [RecordBatch]. +#[derive(Debug, Clone)] +pub struct CollectExec { + plan: Arc, +} + +impl CollectExec { + pub fn new(plan: Arc) -> Self { + Self { plan } + } +} + +#[async_trait] +impl ExecutionPlan for CollectExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.plan.schema() + } + + fn output_partitioning(&self) -> Partitioning { + Partitioning::UnknownPartitioning(1) + } + + fn children(&self) -> Vec> { + vec![self.plan.clone()] + } + + fn with_new_children( + &self, + _children: Vec>, + ) -> Result> { + unimplemented!() + } + + async fn execute( + &self, + partition: usize, + ) -> Result>> { + assert_eq!(0, partition); + let num_partitions = self.plan.output_partitioning().partition_count(); + + let mut futures = Vec::with_capacity(num_partitions); + for i in 0..num_partitions { + futures.push(self.plan.execute(i)); + } + + let mut streams = Vec::with_capacity(num_partitions); + for result in futures::future::join_all(futures).await { + match result { + Ok(stream) => { + streams.push(stream); + } + Err(e) => { + return Err(DataFusionError::Execution(format!( + "BallistaError: {:?}", + e + ))); + } + } + } + + Ok(Box::pin(MergedRecordBatchStream { + schema: self.schema(), + select_all: Box::pin(futures::stream::select_all(streams)), + })) + } +} + +struct MergedRecordBatchStream { + schema: SchemaRef, + select_all: Pin>>, +} + +impl Stream for MergedRecordBatchStream { + type Item = ArrowResult; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + self.select_all.as_mut().poll_next(cx) + } +} + +impl RecordBatchStream for MergedRecordBatchStream { + fn schema(&self) -> SchemaRef { + self.schema.clone() + } +} diff --git a/rust/ballista/rust/executor/src/execution_loop.rs b/rust/ballista/rust/executor/src/execution_loop.rs new file mode 100644 index 0000000000000..5be236f82e0b6 --- /dev/null +++ b/rust/ballista/rust/executor/src/execution_loop.rs @@ -0,0 +1,166 @@ +// 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. + +use std::convert::TryInto; +use std::sync::atomic::{AtomicUsize, Ordering}; +use std::sync::mpsc::{Receiver, Sender, TryRecvError}; +use std::{sync::Arc, time::Duration}; + +use datafusion::physical_plan::ExecutionPlan; +use log::{debug, error, info, warn}; +use tonic::transport::Channel; + +use ballista_core::serde::scheduler::ExecutorMeta; +use ballista_core::{ + client::BallistaClient, + serde::protobuf::{ + self, scheduler_grpc_client::SchedulerGrpcClient, task_status, FailedTask, PartitionId, + PollWorkParams, PollWorkResult, TaskDefinition, TaskStatus, + }, +}; +use protobuf::CompletedTask; + +pub async fn poll_loop( + mut scheduler: SchedulerGrpcClient, + executor_client: BallistaClient, + executor_meta: ExecutorMeta, + concurrent_tasks: usize, +) { + let executor_meta: protobuf::ExecutorMetadata = executor_meta.into(); + let available_tasks_slots = Arc::new(AtomicUsize::new(concurrent_tasks)); + let (task_status_sender, mut task_status_receiver) = std::sync::mpsc::channel::(); + + loop { + debug!("Starting registration loop with scheduler"); + + let task_status: Vec = sample_tasks_status(&mut task_status_receiver).await; + + let poll_work_result: anyhow::Result, tonic::Status> = + scheduler + .poll_work(PollWorkParams { + metadata: Some(executor_meta.clone()), + can_accept_task: available_tasks_slots.load(Ordering::SeqCst) > 0, + task_status, + }) + .await; + + let task_status_sender = task_status_sender.clone(); + + match poll_work_result { + Ok(result) => { + if let Some(task) = result.into_inner().task { + run_received_tasks( + executor_client.clone(), + executor_meta.id.clone(), + available_tasks_slots.clone(), + task_status_sender, + task, + ) + .await; + } + } + Err(error) => { + warn!("Executor registration failed. If this continues to happen the executor might be marked as dead by the scheduler. Error: {}", error); + } + } + + tokio::time::sleep(Duration::from_millis(250)).await; + } +} + +async fn run_received_tasks( + mut executor_client: BallistaClient, + executor_id: String, + available_tasks_slots: Arc, + task_status_sender: Sender, + task: TaskDefinition, +) { + info!("Received task {:?}", task.task_id.as_ref().unwrap()); + available_tasks_slots.fetch_sub(1, Ordering::SeqCst); + let plan: Arc = (&task.plan.unwrap()).try_into().unwrap(); + let task_id = task.task_id.unwrap(); + // TODO: This is a convoluted way of executing the task. We should move the task + // execution code outside of the FlightService (data plane) into the control plane. + + tokio::spawn(async move { + let execution_result = executor_client + .execute_partition( + task_id.job_id.clone(), + task_id.stage_id as usize, + vec![task_id.partition_id as usize], + plan, + ) + .await; + info!("DONE WITH TASK: {:?}", execution_result); + available_tasks_slots.fetch_add(1, Ordering::SeqCst); + let _ = task_status_sender.send(as_task_status( + execution_result.map(|_| ()), + executor_id, + task_id, + )); + }); +} + +fn as_task_status( + execution_result: ballista_core::error::Result<()>, + executor_id: String, + task_id: PartitionId, +) -> TaskStatus { + match execution_result { + Ok(_) => { + info!("Task {:?} finished", task_id); + + TaskStatus { + partition_id: Some(task_id), + status: Some(task_status::Status::Completed(CompletedTask { + executor_id, + })), + } + } + Err(e) => { + let error_msg = e.to_string(); + info!("Task {:?} failed: {}", task_id, error_msg); + + TaskStatus { + partition_id: Some(task_id), + status: Some(task_status::Status::Failed(FailedTask { + error: format!("Task failed due to Tokio error: {}", error_msg), + })), + } + } + } +} + +async fn sample_tasks_status(task_status_receiver: &mut Receiver) -> Vec { + let mut task_status: Vec = vec![]; + + loop { + match task_status_receiver.try_recv() { + anyhow::Result::Ok(status) => { + task_status.push(status); + } + Err(TryRecvError::Empty) => { + break; + } + Err(TryRecvError::Disconnected) => { + error!("Task statuses channel disconnected"); + } + } + } + + task_status +} diff --git a/rust/ballista/rust/executor/src/flight_service.rs b/rust/ballista/rust/executor/src/flight_service.rs new file mode 100644 index 0000000000000..f29bce654c3d0 --- /dev/null +++ b/rust/ballista/rust/executor/src/flight_service.rs @@ -0,0 +1,357 @@ +// 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. + +//! Implementation of the Apache Arrow Flight protocol that wraps an executor. + +use std::fs::File; +use std::path::PathBuf; +use std::pin::Pin; +use std::sync::Arc; +use std::time::Instant; + +use crate::BallistaExecutor; +use ballista_core::error::BallistaError; +use ballista_core::serde::decode_protobuf; +use ballista_core::serde::scheduler::{Action as BallistaAction, PartitionStats}; +use ballista_core::utils::{self, format_plan}; + +use arrow::array::{ArrayRef, StringBuilder}; +use arrow::datatypes::{DataType, Field, Schema}; +use arrow::error::ArrowError; +use arrow::ipc::reader::FileReader; +use arrow::ipc::writer::IpcWriteOptions; +use arrow::record_batch::RecordBatch; +use arrow_flight::{ + flight_service_server::FlightService, Action, ActionType, Criteria, Empty, FlightData, + FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse, PutResult, SchemaResult, + Ticket, +}; +use datafusion::error::DataFusionError; +use futures::{Stream, StreamExt}; +use log::{info, warn}; +use std::io::{Read, Seek}; +use tokio::sync::mpsc::channel; +use tokio::task::JoinHandle; +use tokio::{ + sync::mpsc::{Receiver, Sender}, + task, +}; +use tokio_stream::wrappers::ReceiverStream; +use tonic::{Request, Response, Status, Streaming}; + +type FlightDataSender = Sender>; +type FlightDataReceiver = Receiver>; + +/// Service implementing the Apache Arrow Flight Protocol +#[derive(Clone)] +pub struct BallistaFlightService { + executor: Arc, +} + +impl BallistaFlightService { + pub fn new(executor: Arc) -> Self { + Self { executor } + } +} + +type BoxedFlightStream = Pin> + Send + Sync + 'static>>; + +#[tonic::async_trait] +impl FlightService for BallistaFlightService { + type DoActionStream = BoxedFlightStream; + type DoExchangeStream = BoxedFlightStream; + type DoGetStream = BoxedFlightStream; + type DoPutStream = BoxedFlightStream; + type HandshakeStream = BoxedFlightStream; + type ListActionsStream = BoxedFlightStream; + type ListFlightsStream = BoxedFlightStream; + + async fn do_get( + &self, + request: Request, + ) -> Result, Status> { + let ticket = request.into_inner(); + info!("Received do_get request"); + + let action = decode_protobuf(&ticket.ticket).map_err(|e| from_ballista_err(&e))?; + + match &action { + BallistaAction::ExecutePartition(partition) => { + info!( + "ExecutePartition: job={}, stage={}, partition={:?}\n{}", + partition.job_id, + partition.stage_id, + partition.partition_id, + format_plan(partition.plan.as_ref(), 0).map_err(|e| from_ballista_err(&e))? + ); + + let mut tasks: Vec>> = vec![]; + for part in partition.partition_id.clone() { + let work_dir = self.executor.config.work_dir.clone(); + let partition = partition.clone(); + tasks.push(tokio::spawn(async move { + let mut path = PathBuf::from(&work_dir); + path.push(partition.job_id); + path.push(&format!("{}", partition.stage_id)); + path.push(&format!("{}", part)); + std::fs::create_dir_all(&path)?; + + path.push("data.arrow"); + let path = path.to_str().unwrap(); + info!("Writing results to {}", path); + + let now = Instant::now(); + + // execute the query partition + let mut stream = partition + .plan + .execute(part) + .await + .map_err(|e| from_datafusion_err(&e))?; + + // stream results to disk + let stats = utils::write_stream_to_disk(&mut stream, &path) + .await + .map_err(|e| from_ballista_err(&e))?; + + info!( + "Executed partition {} in {} seconds. Statistics: {:?}", + part, + now.elapsed().as_secs(), + stats + ); + + let mut flights: Vec> = vec![]; + let options = arrow::ipc::writer::IpcWriteOptions::default(); + + let schema = Arc::new(Schema::new(vec![ + Field::new("path", DataType::Utf8, false), + stats.arrow_struct_repr(), + ])); + + // build result set with summary of the partition execution status + let mut c0 = StringBuilder::new(1); + c0.append_value(&path).unwrap(); + let path: ArrayRef = Arc::new(c0.finish()); + + let stats: ArrayRef = stats.to_arrow_arrayref()?; + let results = + vec![RecordBatch::try_new(schema, vec![path, stats]).unwrap()]; + + let mut batches: Vec> = results + .iter() + .flat_map(|batch| create_flight_iter(batch, &options)) + .collect(); + + // append batch vector to schema vector, so that the first message sent is the schema + flights.append(&mut batches); + + Ok(flights) + })); + } + + // wait for all partitions to complete + let results = futures::future::join_all(tasks).await; + + // get results + let mut flights: Vec> = vec![]; + + // add an initial FlightData message that sends schema + let options = arrow::ipc::writer::IpcWriteOptions::default(); + let stats = PartitionStats::default(); + let schema = Arc::new(Schema::new(vec![ + Field::new("path", DataType::Utf8, false), + stats.arrow_struct_repr(), + ])); + let schema_flight_data = + arrow_flight::utils::flight_data_from_arrow_schema(schema.as_ref(), &options); + flights.push(Ok(schema_flight_data)); + + // collect statistics from each executed partition + for result in results { + let result = + result.map_err(|e| Status::internal(format!("Ballista Error: {:?}", e)))?; + let batches = + result.map_err(|e| Status::internal(format!("Ballista Error: {:?}", e)))?; + flights.extend_from_slice(&batches); + } + + let output = futures::stream::iter(flights); + Ok(Response::new(Box::pin(output) as Self::DoGetStream)) + } + BallistaAction::FetchPartition(partition_id) => { + // fetch a partition that was previously executed by this executor + info!("FetchPartition {:?}", partition_id); + + let mut path = PathBuf::from(&self.executor.config.work_dir); + path.push(&partition_id.job_id); + path.push(&format!("{}", partition_id.stage_id)); + path.push(&format!("{}", partition_id.partition_id)); + path.push("data.arrow"); + let path = path.to_str().unwrap(); + + info!("FetchPartition {:?} reading {}", partition_id, path); + let file = File::open(&path) + .map_err(|e| { + BallistaError::General(format!( + "Failed to open partition file at {}: {:?}", + path, e + )) + }) + .map_err(|e| from_ballista_err(&e))?; + let reader = FileReader::try_new(file).map_err(|e| from_arrow_err(&e))?; + + let (tx, rx): (FlightDataSender, FlightDataReceiver) = channel(2); + + // Arrow IPC reader does not implement Sync + Send so we need to use a channel + // to communicate + task::spawn(async move { + if let Err(e) = stream_flight_data(reader, tx).await { + warn!("Error streaming results: {:?}", e); + } + }); + + Ok(Response::new( + Box::pin(ReceiverStream::new(rx)) as Self::DoGetStream + )) + } + } + } + + async fn get_schema( + &self, + _request: Request, + ) -> Result, Status> { + Err(Status::unimplemented("get_schema")) + } + + async fn get_flight_info( + &self, + _request: Request, + ) -> Result, Status> { + Err(Status::unimplemented("get_flight_info")) + } + + async fn handshake( + &self, + _request: Request>, + ) -> Result, Status> { + Err(Status::unimplemented("handshake")) + } + + async fn list_flights( + &self, + _request: Request, + ) -> Result, Status> { + Err(Status::unimplemented("list_flights")) + } + + async fn do_put( + &self, + request: Request>, + ) -> Result, Status> { + let mut request = request.into_inner(); + + while let Some(data) = request.next().await { + let _data = data?; + } + + Err(Status::unimplemented("do_put")) + } + + async fn do_action( + &self, + request: Request, + ) -> Result, Status> { + let action = request.into_inner(); + + let _action = decode_protobuf(&action.body.to_vec()).map_err(|e| from_ballista_err(&e))?; + + Err(Status::unimplemented("do_action")) + } + + async fn list_actions( + &self, + _request: Request, + ) -> Result, Status> { + Err(Status::unimplemented("list_actions")) + } + + async fn do_exchange( + &self, + _request: Request>, + ) -> Result, Status> { + Err(Status::unimplemented("do_exchange")) + } +} + +/// Convert a single RecordBatch into an iterator of FlightData (containing +/// dictionaries and batches) +fn create_flight_iter( + batch: &RecordBatch, + options: &IpcWriteOptions, +) -> Box>> { + let (flight_dictionaries, flight_batch) = + arrow_flight::utils::flight_data_from_arrow_batch(batch, &options); + Box::new( + flight_dictionaries + .into_iter() + .chain(std::iter::once(flight_batch)) + .map(Ok), + ) +} + +async fn stream_flight_data(reader: FileReader, tx: FlightDataSender) -> Result<(), Status> +where + T: Read + Seek, +{ + let options = arrow::ipc::writer::IpcWriteOptions::default(); + let schema_flight_data = + arrow_flight::utils::flight_data_from_arrow_schema(reader.schema().as_ref(), &options); + send_response(&tx, Ok(schema_flight_data)).await?; + + for batch in reader { + let batch_flight_data: Vec<_> = batch + .map(|b| create_flight_iter(&b, &options).collect()) + .map_err(|e| from_arrow_err(&e))?; + for batch in &batch_flight_data { + send_response(&tx, batch.clone()).await?; + } + } + Ok(()) +} + +async fn send_response( + tx: &FlightDataSender, + data: Result, +) -> Result<(), Status> { + tx.send(data) + .await + .map_err(|e| Status::internal(format!("{:?}", e))) +} + +fn from_arrow_err(e: &ArrowError) -> Status { + Status::internal(format!("ArrowError: {:?}", e)) +} + +fn from_ballista_err(e: &ballista_core::error::BallistaError) -> Status { + Status::internal(format!("Ballista Error: {:?}", e)) +} + +fn from_datafusion_err(e: &DataFusionError) -> Status { + Status::internal(format!("DataFusion Error: {:?}", e)) +} diff --git a/rust/ballista/rust/executor/src/lib.rs b/rust/ballista/rust/executor/src/lib.rs new file mode 100644 index 0000000000000..3d7bbaca3f1f0 --- /dev/null +++ b/rust/ballista/rust/executor/src/lib.rs @@ -0,0 +1,52 @@ +// 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. + +//! Core executor logic for executing queries and storing results in memory. + +pub mod collect; +pub mod flight_service; + +#[derive(Debug, Clone)] +pub struct ExecutorConfig { + pub(crate) host: String, + pub(crate) port: u16, + /// Directory for temporary files, such as IPC files + pub(crate) work_dir: String, + pub(crate) concurrent_tasks: usize, +} + +impl ExecutorConfig { + pub fn new(host: &str, port: u16, work_dir: &str, concurrent_tasks: usize) -> Self { + Self { + host: host.to_owned(), + port, + work_dir: work_dir.to_owned(), + concurrent_tasks, + } + } +} + +#[allow(dead_code)] +pub struct BallistaExecutor { + pub(crate) config: ExecutorConfig, +} + +impl BallistaExecutor { + pub fn new(config: ExecutorConfig) -> Self { + Self { config } + } +} diff --git a/rust/ballista/rust/executor/src/main.rs b/rust/ballista/rust/executor/src/main.rs new file mode 100644 index 0000000000000..7e7877a0ce85c --- /dev/null +++ b/rust/ballista/rust/executor/src/main.rs @@ -0,0 +1,168 @@ +// 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 Rust executor binary. + +use std::sync::Arc; + +use anyhow::{Context, Result}; +use arrow_flight::flight_service_server::FlightServiceServer; +use futures::future::MaybeDone; +use log::info; +use tempfile::TempDir; +use tonic::transport::Server; +use uuid::Uuid; + +use ballista_core::{ + client::BallistaClient, serde::protobuf::scheduler_grpc_client::SchedulerGrpcClient, +}; +use ballista_core::{ + print_version, serde::protobuf::scheduler_grpc_server::SchedulerGrpcServer, + serde::scheduler::ExecutorMeta, BALLISTA_VERSION, +}; +use ballista_executor::{flight_service::BallistaFlightService, BallistaExecutor, ExecutorConfig}; +use ballista_scheduler::{state::StandaloneClient, SchedulerServer}; +use config::prelude::*; + +mod execution_loop; + +#[macro_use] +extern crate configure_me; + +#[allow(clippy::all, warnings)] +mod config { + // Ideally we would use the include_config macro from configure_me, but then we cannot use + // #[allow(clippy::all)] to silence clippy warnings from the generated code + include!(concat!(env!("OUT_DIR"), "/executor_configure_me_config.rs")); +} + +#[cfg(feature = "snmalloc")] +#[global_allocator] +static ALLOC: snmalloc_rs::SnMalloc = snmalloc_rs::SnMalloc; + +#[tokio::main] +async fn main() -> Result<()> { + env_logger::init(); + + // parse command-line arguments + let (opt, _remaining_args) = + Config::including_optional_config_files(&["/etc/ballista/executor.toml"]).unwrap_or_exit(); + + if opt.version { + print_version(); + std::process::exit(0); + } + + let namespace = opt.namespace; + let external_host = opt.external_host; + let bind_host = opt.bind_host; + let port = opt.port; + + let addr = format!("{}:{}", bind_host, port); + let addr = addr + .parse() + .with_context(|| format!("Could not parse address: {}", addr))?; + + let scheduler_host = if opt.local { + external_host.to_owned() + } else { + opt.scheduler_host + }; + let scheduler_port = opt.scheduler_port; + let scheduler_url = format!("http://{}:{}", scheduler_host, scheduler_port); + + let work_dir = opt.work_dir.unwrap_or( + TempDir::new()? + .into_path() + .into_os_string() + .into_string() + .unwrap(), + ); + let config = ExecutorConfig::new(&external_host, port, &work_dir, opt.concurrent_tasks); + info!("Running with config: {:?}", config); + + let executor_meta = ExecutorMeta { + id: Uuid::new_v4().to_string(), // assign this executor a unique ID + host: external_host.clone(), + port, + }; + + if opt.local { + info!("Running in local mode. Scheduler will be run in-proc"); + let client = StandaloneClient::try_new_temporary() + .context("Could not create standalone config backend")?; + let server = SchedulerGrpcServer::new(SchedulerServer::new(Arc::new(client), namespace)); + let addr = format!("{}:{}", bind_host, scheduler_port); + let addr = addr + .parse() + .with_context(|| format!("Could not parse {}", addr))?; + info!( + "Ballista v{} Rust Scheduler listening on {:?}", + BALLISTA_VERSION, addr + ); + let scheduler_future = tokio::spawn(Server::builder().add_service(server).serve(addr)); + let mut scheduler_result = futures::future::maybe_done(scheduler_future); + + // Ensure scheduler is ready to receive connections + while SchedulerGrpcClient::connect(scheduler_url.clone()) + .await + .is_err() + { + let scheduler_future = match scheduler_result { + MaybeDone::Future(f) => f, + MaybeDone::Done(Err(e)) => return Err(e).context("Tokio error"), + MaybeDone::Done(Ok(Err(e))) => { + return Err(e).context("Scheduler failed to initialize correctly") + } + MaybeDone::Done(Ok(Ok(()))) => { + return Err(anyhow::format_err!( + "Scheduler unexpectedly finished successfully" + )) + } + MaybeDone::Gone => panic!("Received Gone from recently created MaybeDone"), + }; + tokio::time::sleep(std::time::Duration::from_millis(100)).await; + scheduler_result = futures::future::maybe_done(scheduler_future); + } + } + + let scheduler = SchedulerGrpcClient::connect(scheduler_url) + .await + .context("Could not connect to scheduler")?; + let executor = Arc::new(BallistaExecutor::new(config)); + let service = BallistaFlightService::new(executor); + + let server = FlightServiceServer::new(service); + info!( + "Ballista v{} Rust Executor listening on {:?}", + BALLISTA_VERSION, addr + ); + let server_future = tokio::spawn(Server::builder().add_service(server).serve(addr)); + let client = BallistaClient::try_new(&external_host, port).await?; + tokio::spawn(execution_loop::poll_loop( + scheduler, + client, + executor_meta, + opt.concurrent_tasks, + )); + + server_future + .await + .context("Tokio error")? + .context("Could not start executor server")?; + Ok(()) +} diff --git a/rust/ballista/rust/scheduler/Cargo.toml b/rust/ballista/rust/scheduler/Cargo.toml new file mode 100644 index 0000000000000..525e28a63cc88 --- /dev/null +++ b/rust/ballista/rust/scheduler/Cargo.toml @@ -0,0 +1,61 @@ +# 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. + +[package] +name = "ballista-scheduler" +description = "Ballista Distributed Compute - Scheduler" +license = "Apache-2.0" +version = "0.4.2-SNAPSHOT" +homepage = "https://github.com/apache/arrow" +repository = "https://github.com/apache/arrow" +authors = ["Apache Arrow "] +edition = "2018" + +[features] +default = ["etcd", "sled"] +etcd = ["etcd-client"] +sled = ["sled_package"] + +[dependencies] +anyhow = "1" +ballista-core = { path = "../core" } +clap = "2" +configure_me = "0.4.0" +env_logger = "0.8" +etcd-client = { version = "0.6", optional = true } +futures = "0.3" +log = "0.4" +parse_arg = "0.1.3" +prost = "0.7" +rand = "0.8" +serde = {version = "1", features = ["derive"]} +sled_package = { package = "sled", version = "0.34", optional = true } +tokio = { version = "1.0", features = ["macros", "rt", "rt-multi-thread"] } +tonic = "0.4" + +arrow = { git = "https://github.com/apache/arrow", rev="46161d2" } +datafusion = { git = "https://github.com/apache/arrow", rev="46161d2" } + +[dev-dependencies] +ballista-core = { path = "../core" } +uuid = { version = "0.8", features = ["v4"] } + +[build-dependencies] +configure_me_codegen = "0.4.0" + +[package.metadata.configure_me.bin] +scheduler = "scheduler_config_spec.toml" diff --git a/rust/ballista/rust/scheduler/README.md b/rust/ballista/rust/scheduler/README.md new file mode 100644 index 0000000000000..facc6d176982c --- /dev/null +++ b/rust/ballista/rust/scheduler/README.md @@ -0,0 +1,32 @@ + + +# Ballista Scheduler +This crate contains the Ballista Scheduler. It can be used both as a library or as a binary. + +## Run + +```bash +$ RUST_LOG=info cargo run --release +... +[2021-02-11T05:29:30Z INFO scheduler] Ballista v0.4.2-SNAPSHOT Scheduler listening on 0.0.0.0:50050 +[2021-02-11T05:30:13Z INFO ballista::scheduler] Received register_executor request for ExecutorMetadata { id: "6d10f5d2-c8c3-4e0f-afdb-1f6ec9171321", host: "localhost", port: 50051 } +``` + +By default, the scheduler will bind to `localhost` and listen on port `50051`. diff --git a/rust/ballista/rust/scheduler/build.rs b/rust/ballista/rust/scheduler/build.rs new file mode 100644 index 0000000000000..bae6a3bfe2e60 --- /dev/null +++ b/rust/ballista/rust/scheduler/build.rs @@ -0,0 +1,24 @@ +// 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. + +extern crate configure_me_codegen; + +fn main() -> Result<(), String> { + println!("cargo:rerun-if-changed=scheduler_config_spec.toml"); + configure_me_codegen::build_script_auto() + .map_err(|e| format!("configure_me code generation failed: {}", e)) +} diff --git a/rust/ballista/rust/scheduler/scheduler_config_spec.toml b/rust/ballista/rust/scheduler/scheduler_config_spec.toml new file mode 100644 index 0000000000000..560e9a2599bd5 --- /dev/null +++ b/rust/ballista/rust/scheduler/scheduler_config_spec.toml @@ -0,0 +1,60 @@ +# 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. + +[general] +name = "Ballista Scheduler" +env_prefix = "BALLISTA_SCHEDULER" +conf_file_param = "config_file" + +[[switch]] +name = "version" +doc = "Print version of this executable" + +[[param]] +abbr = "b" +name = "config_backend" +type = "ballista_scheduler::ConfigBackend" +doc = "The configuration backend for the scheduler, see ConfigBackend::variants() for options. Default: Standalone" +default = "ballista_scheduler::ConfigBackend::Standalone" + +[[param]] +abbr = "n" +name = "namespace" +type = "String" +doc = "Namespace for the ballista cluster that this executor will join. Default: ballista" +default = "std::string::String::from(\"ballista\")" + +[[param]] +abbr = "e" +name = "etcd_urls" +type = "String" +doc = "etcd urls for use when discovery mode is `etcd`. Default: localhost:2379" +default = "std::string::String::from(\"localhost:2379\")" + +[[param]] +abbr = "h" +name = "bind_host" +type = "String" +default = "std::string::String::from(\"0.0.0.0\")" +doc = "Local host name or IP address to bind to. Default: 0.0.0.0" + +[[param]] +abbr = "p" +name = "port" +type = "u16" +default = "50050" +doc = "bind port. Default: 50050" \ No newline at end of file diff --git a/rust/ballista/rust/scheduler/src/lib.rs b/rust/ballista/rust/scheduler/src/lib.rs new file mode 100644 index 0000000000000..39c56b680db15 --- /dev/null +++ b/rust/ballista/rust/scheduler/src/lib.rs @@ -0,0 +1,495 @@ +// 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. + +//! Support for distributed schedulers, such as Kubernetes + +pub mod planner; +pub mod state; + +#[cfg(test)] +pub mod test_utils; + +use std::fmt; +use std::{convert::TryInto, sync::Arc}; + +use ballista_core::serde::protobuf::{ + execute_query_params::Query, job_status, scheduler_grpc_server::SchedulerGrpc, + ExecuteQueryParams, ExecuteQueryResult, FailedJob, FilePartitionMetadata, FileType, + GetExecutorMetadataParams, GetExecutorMetadataResult, GetFileMetadataParams, + GetFileMetadataResult, GetJobStatusParams, GetJobStatusResult, JobStatus, PartitionId, + PollWorkParams, PollWorkResult, QueuedJob, RunningJob, TaskDefinition, TaskStatus, +}; +use ballista_core::serde::scheduler::ExecutorMeta; + +use clap::arg_enum; +use datafusion::physical_plan::ExecutionPlan; +#[cfg(feature = "sled")] +extern crate sled_package as sled; + +// an enum used to configure the backend +// needs to be visible to code generated by configure_me +arg_enum! { + #[derive(Debug, serde::Deserialize)] + pub enum ConfigBackend { + Etcd, + Standalone + } +} + +impl parse_arg::ParseArgFromStr for ConfigBackend { + fn describe_type(mut writer: W) -> fmt::Result { + write!(writer, "The configuration backend for the scheduler") + } +} + +use crate::planner::DistributedPlanner; + +use datafusion::execution::context::ExecutionContext; +use log::{debug, error, info, warn}; +use rand::{distributions::Alphanumeric, thread_rng, Rng}; +use tonic::{Request, Response}; + +use self::state::{ConfigBackendClient, SchedulerState}; +use datafusion::physical_plan::parquet::ParquetExec; +use std::time::Instant; + +pub struct SchedulerServer { + state: SchedulerState, + namespace: String, +} + +impl SchedulerServer { + pub fn new(config: Arc, namespace: String) -> Self { + Self { + state: SchedulerState::new(config), + namespace, + } + } +} + +#[tonic::async_trait] +impl SchedulerGrpc for SchedulerServer { + async fn get_executors_metadata( + &self, + _request: Request, + ) -> std::result::Result, tonic::Status> { + info!("Received get_executors_metadata request"); + let result = self + .state + .get_executors_metadata(self.namespace.as_str()) + .await + .map_err(|e| { + let msg = format!("Error reading executors metadata: {}", e); + error!("{}", msg); + tonic::Status::internal(msg) + })? + .into_iter() + .map(|meta| meta.into()) + .collect(); + Ok(Response::new(GetExecutorMetadataResult { + metadata: result, + })) + } + + async fn poll_work( + &self, + request: Request, + ) -> std::result::Result, tonic::Status> { + if let PollWorkParams { + metadata: Some(metadata), + can_accept_task, + task_status, + } = request.into_inner() + { + debug!("Received poll_work request for {:?}", metadata); + let metadata: ExecutorMeta = metadata.into(); + let mut lock = self.state.lock().await.map_err(|e| { + let msg = format!("Could not lock the state: {}", e); + error!("{}", msg); + tonic::Status::internal(msg) + })?; + self.state + .save_executor_metadata(&self.namespace, metadata.clone()) + .await + .map_err(|e| { + let msg = format!("Could not save executor metadata: {}", e); + error!("{}", msg); + tonic::Status::internal(msg) + })?; + let task_status_empty = task_status.is_empty(); + for task_status in task_status { + self.state + .save_task_status(&self.namespace, &task_status) + .await + .map_err(|e| { + let msg = format!("Could not save task status: {}", e); + error!("{}", msg); + tonic::Status::internal(msg) + })?; + } + let task = if can_accept_task { + let plan = self + .state + .assign_next_schedulable_task(&self.namespace, &metadata.id) + .await + .map_err(|e| { + let msg = format!("Error finding next assignable task: {}", e); + error!("{}", msg); + tonic::Status::internal(msg) + })?; + if let Some((task, _plan)) = &plan { + let partition_id = task.partition_id.as_ref().unwrap(); + info!( + "Sending new task to {}: {}/{}/{}", + metadata.id, + partition_id.job_id, + partition_id.stage_id, + partition_id.partition_id + ); + } + plan.map(|(status, plan)| TaskDefinition { + plan: Some(plan.try_into().unwrap()), + task_id: status.partition_id, + }) + } else { + None + }; + // TODO: this should probably happen asynchronously with a watch on etc/sled + if !task_status_empty { + if let Err(e) = self.state.synchronize_job_status(&self.namespace).await { + warn!("Could not synchronize jobs and tasks state: {}", e); + } + } + lock.unlock().await; + Ok(Response::new(PollWorkResult { task })) + } else { + warn!("Received invalid executor poll_work request"); + Err(tonic::Status::invalid_argument( + "Missing metadata in request", + )) + } + } + + async fn get_file_metadata( + &self, + request: Request, + ) -> std::result::Result, tonic::Status> { + let GetFileMetadataParams { path, file_type } = request.into_inner(); + + let file_type: FileType = file_type.try_into().map_err(|e| { + let msg = format!("Error reading request: {}", e); + error!("{}", msg); + tonic::Status::internal(msg) + })?; + + match file_type { + FileType::Parquet => { + let parquet_exec = + ParquetExec::try_from_path(&path, None, None, 1024, 1).map_err(|e| { + let msg = format!("Error opening parquet files: {}", e); + error!("{}", msg); + tonic::Status::internal(msg) + })?; + + //TODO include statistics and any other info needed to reconstruct ParquetExec + Ok(Response::new(GetFileMetadataResult { + schema: Some(parquet_exec.schema().as_ref().into()), + partitions: parquet_exec + .partitions() + .iter() + .map(|part| FilePartitionMetadata { + filename: part.filenames().to_vec(), + }) + .collect(), + })) + } + //TODO implement for CSV + _ => Err(tonic::Status::unimplemented( + "get_file_metadata unsupported file type", + )), + } + } + + async fn execute_query( + &self, + request: Request, + ) -> std::result::Result, tonic::Status> { + if let ExecuteQueryParams { query: Some(query) } = request.into_inner() { + let plan = match query { + Query::LogicalPlan(logical_plan) => { + // parse protobuf + (&logical_plan).try_into().map_err(|e| { + let msg = format!("Could not parse logical plan protobuf: {}", e); + error!("{}", msg); + tonic::Status::internal(msg) + })? + } + Query::Sql(sql) => { + //TODO we can't just create a new context because we need a context that has + // tables registered from previous SQL statements that have been executed + let mut ctx = ExecutionContext::new(); + let df = ctx.sql(&sql).map_err(|e| { + let msg = format!("Error parsing SQL: {}", e); + error!("{}", msg); + tonic::Status::internal(msg) + })?; + df.to_logical_plan() + } + }; + debug!("Received plan for execution: {:?}", plan); + let executors = self + .state + .get_executors_metadata(&self.namespace) + .await + .map_err(|e| { + let msg = format!("Error reading executors metadata: {}", e); + error!("{}", msg); + tonic::Status::internal(msg) + })?; + debug!("Found executors: {:?}", executors); + + let job_id: String = { + let mut rng = thread_rng(); + std::iter::repeat(()) + .map(|()| rng.sample(Alphanumeric)) + .map(char::from) + .take(7) + .collect() + }; + + // Save placeholder job metadata + self.state + .save_job_metadata( + &self.namespace, + &job_id, + &JobStatus { + status: Some(job_status::Status::Queued(QueuedJob {})), + }, + ) + .await + .map_err(|e| { + tonic::Status::internal(format!("Could not save job metadata: {}", e)) + })?; + + let namespace = self.namespace.to_owned(); + let state = self.state.clone(); + let job_id_spawn = job_id.clone(); + tokio::spawn(async move { + // create physical plan using DataFusion + let datafusion_ctx = ExecutionContext::new(); + macro_rules! fail_job { + ($code :expr) => {{ + match $code { + Err(error) => { + warn!("Job {} failed with {}", job_id_spawn, error); + state + .save_job_metadata( + &namespace, + &job_id_spawn, + &JobStatus { + status: Some(job_status::Status::Failed(FailedJob { + error: format!("{}", error), + })), + }, + ) + .await + .unwrap(); + return; + } + Ok(value) => value, + } + }}; + }; + + let start = Instant::now(); + + let optimized_plan = fail_job!(datafusion_ctx.optimize(&plan).map_err(|e| { + let msg = format!("Could not create optimized logical plan: {}", e); + error!("{}", msg); + tonic::Status::internal(msg) + })); + + debug!("Calculated optimized plan: {:?}", optimized_plan); + + let plan = fail_job!(datafusion_ctx + .create_physical_plan(&optimized_plan) + .map_err(|e| { + let msg = format!("Could not create physical plan: {}", e); + error!("{}", msg); + tonic::Status::internal(msg) + })); + + info!( + "DataFusion created physical plan in {} milliseconds", + start.elapsed().as_millis(), + ); + + // create distributed physical plan using Ballista + if let Err(e) = state + .save_job_metadata( + &namespace, + &job_id_spawn, + &JobStatus { + status: Some(job_status::Status::Running(RunningJob {})), + }, + ) + .await + { + warn!( + "Could not update job {} status to running: {}", + job_id_spawn, e + ); + } + let mut planner = fail_job!(DistributedPlanner::try_new(executors).map_err(|e| { + let msg = format!("Could not create distributed planner: {}", e); + error!("{}", msg); + tonic::Status::internal(msg) + })); + let stages = + fail_job!(planner.plan_query_stages(&job_id_spawn, plan).map_err(|e| { + let msg = format!("Could not plan query stages: {}", e); + error!("{}", msg); + tonic::Status::internal(msg) + })); + + // save stages into state + for stage in stages { + fail_job!(state + .save_stage_plan( + &namespace, + &job_id_spawn, + stage.stage_id, + stage.child.clone() + ) + .await + .map_err(|e| { + let msg = format!("Could not save stage plan: {}", e); + error!("{}", msg); + tonic::Status::internal(msg) + })); + let num_partitions = stage.output_partitioning().partition_count(); + for partition_id in 0..num_partitions { + let pending_status = TaskStatus { + partition_id: Some(PartitionId { + job_id: job_id_spawn.clone(), + stage_id: stage.stage_id as u32, + partition_id: partition_id as u32, + }), + status: None, + }; + fail_job!(state + .save_task_status(&namespace, &pending_status) + .await + .map_err(|e| { + let msg = format!("Could not save task status: {}", e); + error!("{}", msg); + tonic::Status::internal(msg) + })); + } + } + }); + + Ok(Response::new(ExecuteQueryResult { job_id })) + } else { + Err(tonic::Status::internal("Error parsing request")) + } + } + + async fn get_job_status( + &self, + request: Request, + ) -> std::result::Result, tonic::Status> { + let job_id = request.into_inner().job_id; + debug!("Received get_job_status request for job {}", job_id); + let job_meta = self + .state + .get_job_metadata(&self.namespace, &job_id) + .await + .map_err(|e| { + let msg = format!("Error reading job metadata: {}", e); + error!("{}", msg); + tonic::Status::internal(msg) + })?; + Ok(Response::new(GetJobStatusResult { + status: Some(job_meta), + })) + } +} + +#[cfg(all(test, feature = "sled"))] +mod test { + use std::sync::Arc; + + use tonic::Request; + + use ballista_core::error::BallistaError; + use ballista_core::serde::protobuf::{ExecutorMetadata, PollWorkParams}; + + use super::{ + state::{SchedulerState, StandaloneClient}, + SchedulerGrpc, SchedulerServer, + }; + + #[tokio::test] + async fn test_poll_work() -> Result<(), BallistaError> { + let state = Arc::new(StandaloneClient::try_new_temporary()?); + let namespace = "default"; + let scheduler = SchedulerServer::new(state.clone(), namespace.to_owned()); + let state = SchedulerState::new(state); + let exec_meta = ExecutorMetadata { + id: "abc".to_owned(), + host: "".to_owned(), + port: 0, + }; + let request: Request = Request::new(PollWorkParams { + metadata: Some(exec_meta.clone()), + can_accept_task: false, + task_status: vec![], + }); + let response = scheduler + .poll_work(request) + .await + .expect("Received error response") + .into_inner(); + // no response task since we told the scheduler we didn't want to accept one + assert!(response.task.is_none()); + // executor should be registered + assert_eq!( + state.get_executors_metadata(namespace).await.unwrap().len(), + 1 + ); + + let request: Request = Request::new(PollWorkParams { + metadata: Some(exec_meta.clone()), + can_accept_task: true, + task_status: vec![], + }); + let response = scheduler + .poll_work(request) + .await + .expect("Received error response") + .into_inner(); + // still no response task since there are no tasks in the scheduelr + assert!(response.task.is_none()); + // executor should be registered + assert_eq!( + state.get_executors_metadata(namespace).await.unwrap().len(), + 1 + ); + Ok(()) + } +} diff --git a/rust/ballista/rust/scheduler/src/main.rs b/rust/ballista/rust/scheduler/src/main.rs new file mode 100644 index 0000000000000..2a569bee51eef --- /dev/null +++ b/rust/ballista/rust/scheduler/src/main.rs @@ -0,0 +1,120 @@ +// 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 Rust scheduler binary. + +use std::{net::SocketAddr, sync::Arc}; + +use anyhow::{Context, Result}; +use ballista_core::BALLISTA_VERSION; +use ballista_core::{print_version, serde::protobuf::scheduler_grpc_server::SchedulerGrpcServer}; +#[cfg(feature = "etcd")] +use ballista_scheduler::state::EtcdClient; +#[cfg(feature = "sled")] +use ballista_scheduler::state::StandaloneClient; +use ballista_scheduler::{state::ConfigBackendClient, ConfigBackend, SchedulerServer}; + +use log::info; +use tonic::transport::Server; + +#[macro_use] +extern crate configure_me; + +#[allow(clippy::all, warnings)] +mod config { + // Ideally we would use the include_config macro from configure_me, but then we cannot use + // #[allow(clippy::all)] to silence clippy warnings from the generated code + include!(concat!( + env!("OUT_DIR"), + "/scheduler_configure_me_config.rs" + )); +} +use config::prelude::*; + +async fn start_server( + config_backend: Arc, + namespace: String, + addr: SocketAddr, +) -> Result<()> { + info!( + "Ballista v{} Scheduler listening on {:?}", + BALLISTA_VERSION, addr + ); + let server = SchedulerGrpcServer::new(SchedulerServer::new(config_backend, namespace)); + Ok(Server::builder() + .add_service(server) + .serve(addr) + .await + .context("Could not start grpc server")?) +} + +#[tokio::main] +async fn main() -> Result<()> { + env_logger::init(); + + // parse options + let (opt, _remaining_args) = + Config::including_optional_config_files(&["/etc/ballista/scheduler.toml"]).unwrap_or_exit(); + + if opt.version { + print_version(); + std::process::exit(0); + } + + let namespace = opt.namespace; + let bind_host = opt.bind_host; + let port = opt.port; + + let addr = format!("{}:{}", bind_host, port); + let addr = addr.parse()?; + + let client: Arc = match opt.config_backend { + #[cfg(not(any(feature = "sled", feature = "etcd")))] + _ => std::compile_error!( + "To build the scheduler enable at least one config backend feature (`etcd` or `sled`)" + ), + #[cfg(feature = "etcd")] + ConfigBackend::Etcd => { + let etcd = etcd_client::Client::connect(&[opt.etcd_urls], None) + .await + .context("Could not connect to etcd")?; + Arc::new(EtcdClient::new(etcd)) + } + #[cfg(not(feature = "etcd"))] + ConfigBackend::Etcd => { + unimplemented!( + "build the scheduler with the `etcd` feature to use the etcd config backend" + ) + } + #[cfg(feature = "sled")] + ConfigBackend::Standalone => { + // TODO: Use a real file and make path is configurable + Arc::new( + StandaloneClient::try_new_temporary() + .context("Could not create standalone config backend")?, + ) + } + #[cfg(not(feature = "sled"))] + ConfigBackend::Standalone => { + unimplemented!( + "build the scheduler with the `sled` feature to use the standalone config backend" + ) + } + }; + start_server(client, namespace, addr).await?; + Ok(()) +} diff --git a/rust/ballista/rust/scheduler/src/planner.rs b/rust/ballista/rust/scheduler/src/planner.rs new file mode 100644 index 0000000000000..414b521c49dd6 --- /dev/null +++ b/rust/ballista/rust/scheduler/src/planner.rs @@ -0,0 +1,473 @@ +// 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. + +//! Distributed query execution +//! +//! This code is EXPERIMENTAL and still under development + +use std::pin::Pin; +use std::sync::Arc; +use std::time::Instant; +use std::{collections::HashMap, future::Future}; + +use ballista_core::client::BallistaClient; +use ballista_core::datasource::DFTableAdapter; +use ballista_core::error::{BallistaError, Result}; +use ballista_core::serde::scheduler::ExecutorMeta; +use ballista_core::serde::scheduler::PartitionId; +use ballista_core::utils::format_plan; +use ballista_core::{ + execution_plans::{QueryStageExec, ShuffleReaderExec, UnresolvedShuffleExec}, + serde::scheduler::PartitionLocation, +}; +use datafusion::execution::context::ExecutionContext; +use datafusion::physical_plan::hash_aggregate::{AggregateMode, HashAggregateExec}; +use datafusion::physical_plan::hash_join::HashJoinExec; +use datafusion::physical_plan::merge::MergeExec; +use datafusion::physical_plan::ExecutionPlan; +use log::{debug, info}; +use tokio::task::JoinHandle; + +type SendableExecutionPlan = Pin>> + Send>>; +type PartialQueryStageResult = (Arc, Vec>); + +pub struct DistributedPlanner { + executors: Vec, + next_stage_id: usize, +} + +impl DistributedPlanner { + pub fn try_new(executors: Vec) -> Result { + if executors.is_empty() { + Err(BallistaError::General( + "DistributedPlanner requires at least one executor".to_owned(), + )) + } else { + Ok(Self { + executors, + next_stage_id: 0, + }) + } + } +} + +impl DistributedPlanner { + /// Execute a distributed query against a cluster, leaving the final results on the + /// executors. The [ExecutionPlan] returned by this method is guaranteed to be a + /// [ShuffleReaderExec] that can be used to fetch the final results from the executors + /// in parallel. + pub async fn execute_distributed_query( + &mut self, + job_id: String, + execution_plan: Arc, + ) -> Result> { + let now = Instant::now(); + let execution_plans = self.plan_query_stages(&job_id, execution_plan)?; + + info!( + "DistributedPlanner created {} execution plans in {} seconds:", + execution_plans.len(), + now.elapsed().as_secs() + ); + + for plan in &execution_plans { + info!("{}", format_plan(plan.as_ref(), 0)?); + } + + execute(execution_plans, self.executors.clone()).await + } + + /// Returns a vector of ExecutionPlans, where the root node is a [QueryStageExec]. + /// Plans that depend on the input of other plans will have leaf nodes of type [UnresolvedShuffleExec]. + /// A [QueryStageExec] is created whenever the partitioning changes. + /// + /// Returns an empty vector if the execution_plan doesn't need to be sliced into several stages. + pub fn plan_query_stages( + &mut self, + job_id: &str, + execution_plan: Arc, + ) -> Result>> { + info!("planning query stages"); + let (new_plan, mut stages) = self.plan_query_stages_internal(job_id, execution_plan)?; + stages.push(create_query_stage( + job_id.to_string(), + self.next_stage_id(), + new_plan, + )?); + Ok(stages) + } + + /// Returns a potentially modified version of the input execution_plan along with the resulting query stages. + /// This function is needed because the input execution_plan might need to be modified, but it might not hold a + /// compelte query stage (its parent might also belong to the same stage) + fn plan_query_stages_internal( + &mut self, + job_id: &str, + execution_plan: Arc, + ) -> Result { + // recurse down and replace children + if execution_plan.children().is_empty() { + return Ok((execution_plan, vec![])); + } + + let mut stages = vec![]; + let mut children = vec![]; + for child in execution_plan.children() { + let (new_child, mut child_stages) = + self.plan_query_stages_internal(job_id, child.clone())?; + children.push(new_child); + stages.append(&mut child_stages); + } + + if let Some(adapter) = execution_plan.as_any().downcast_ref::() { + let ctx = ExecutionContext::new(); + Ok((ctx.create_physical_plan(&adapter.logical_plan)?, stages)) + } else if let Some(merge) = execution_plan.as_any().downcast_ref::() { + let query_stage = create_query_stage( + job_id.to_string(), + self.next_stage_id(), + merge.children()[0].clone(), + )?; + let unresolved_shuffle = Arc::new(UnresolvedShuffleExec::new( + vec![query_stage.stage_id], + query_stage.schema(), + query_stage.output_partitioning().partition_count(), + )); + stages.push(query_stage); + Ok((merge.with_new_children(vec![unresolved_shuffle])?, stages)) + } else if let Some(agg) = execution_plan.as_any().downcast_ref::() { + //TODO should insert query stages in more generic way based on partitioning metadata + // and not specifically for this operator + match agg.mode() { + AggregateMode::Final => { + let mut new_children: Vec> = vec![]; + for child in &children { + let new_stage = create_query_stage( + job_id.to_string(), + self.next_stage_id(), + child.clone(), + )?; + new_children.push(Arc::new(UnresolvedShuffleExec::new( + vec![new_stage.stage_id], + new_stage.schema().clone(), + new_stage.output_partitioning().partition_count(), + ))); + stages.push(new_stage); + } + Ok((agg.with_new_children(new_children)?, stages)) + } + AggregateMode::Partial => Ok((agg.with_new_children(children)?, stages)), + } + } else if let Some(join) = execution_plan.as_any().downcast_ref::() { + Ok((join.with_new_children(children)?, stages)) + } else { + // TODO check for compatible partitioning schema, not just count + if execution_plan.output_partitioning().partition_count() + != children[0].output_partitioning().partition_count() + { + let mut new_children: Vec> = vec![]; + for child in &children { + let new_stage = create_query_stage( + job_id.to_string(), + self.next_stage_id(), + child.clone(), + )?; + new_children.push(Arc::new(UnresolvedShuffleExec::new( + vec![new_stage.stage_id], + new_stage.schema().clone(), + new_stage.output_partitioning().partition_count(), + ))); + stages.push(new_stage); + } + Ok((execution_plan.with_new_children(new_children)?, stages)) + } else { + Ok((execution_plan.with_new_children(children)?, stages)) + } + } + } + + /// Generate a new stage ID + fn next_stage_id(&mut self) -> usize { + self.next_stage_id += 1; + self.next_stage_id + } +} + +fn execute( + stages: Vec>, + executors: Vec, +) -> SendableExecutionPlan { + Box::pin(async move { + let mut partition_locations: HashMap> = HashMap::new(); + let mut result_partition_locations = vec![]; + for stage in &stages { + debug!("execute() {}", &format!("{:?}", stage)[0..60]); + let stage = remove_unresolved_shuffles(stage.as_ref(), &partition_locations)?; + let stage = stage.as_any().downcast_ref::().unwrap(); + result_partition_locations = execute_query_stage( + &stage.job_id.clone(), + stage.stage_id, + stage.children()[0].clone(), + executors.clone(), + ) + .await?; + partition_locations.insert(stage.stage_id, result_partition_locations.clone()); + } + + let shuffle_reader: Arc = Arc::new(ShuffleReaderExec::try_new( + result_partition_locations, + stages.last().unwrap().schema(), + )?); + Ok(shuffle_reader) + }) +} + +pub fn remove_unresolved_shuffles( + stage: &dyn ExecutionPlan, + partition_locations: &HashMap>, +) -> Result> { + let mut new_children: Vec> = vec![]; + for child in stage.children() { + if let Some(unresolved_shuffle) = child.as_any().downcast_ref::() { + let mut relevant_locations = vec![]; + for id in &unresolved_shuffle.query_stage_ids { + relevant_locations.append( + &mut partition_locations + .get(id) + .ok_or_else(|| { + BallistaError::General( + "Missing partition location. Could not remove unresolved shuffles" + .to_owned(), + ) + })? + .clone(), + ); + } + new_children.push(Arc::new(ShuffleReaderExec::try_new( + relevant_locations, + unresolved_shuffle.schema().clone(), + )?)) + } else { + new_children.push(remove_unresolved_shuffles( + child.as_ref(), + partition_locations, + )?); + } + } + Ok(stage.with_new_children(new_children)?) +} + +fn create_query_stage( + job_id: String, + stage_id: usize, + plan: Arc, +) -> Result> { + Ok(Arc::new(QueryStageExec::try_new(job_id, stage_id, plan)?)) +} + +/// Execute a query stage by sending each partition to an executor +async fn execute_query_stage( + job_id: &str, + stage_id: usize, + plan: Arc, + executors: Vec, +) -> Result> { + info!( + "execute_query_stage() stage_id={}\n{}", + stage_id, + format_plan(plan.as_ref(), 0)? + ); + + let partition_count = plan.output_partitioning().partition_count(); + + let num_chunks = partition_count / executors.len(); + let num_chunks = num_chunks.max(1); + let partition_chunks: Vec> = (0..partition_count) + .collect::>() + .chunks(num_chunks) + .map(|r| r.to_vec()) + .collect(); + + info!( + "Executing query stage with {} chunks of partition ranges", + partition_chunks.len() + ); + + let mut executions: Vec>>> = + Vec::with_capacity(partition_count); + for i in 0..partition_chunks.len() { + let plan = plan.clone(); + let executor_meta = executors[i % executors.len()].clone(); + let partition_ids = partition_chunks[i].to_vec(); + let job_id = job_id.to_owned(); + executions.push(tokio::spawn(async move { + let mut client = + BallistaClient::try_new(&executor_meta.host, executor_meta.port).await?; + let stats = client + .execute_partition(job_id.clone(), stage_id, partition_ids.clone(), plan) + .await?; + + Ok(partition_ids + .iter() + .map(|part| PartitionLocation { + partition_id: PartitionId::new(&job_id, stage_id, *part), + executor_meta: executor_meta.clone(), + partition_stats: *stats[*part].statistics(), + }) + .collect()) + })); + } + + // wait for all partitions to complete + let results = futures::future::join_all(executions).await; + + // check for errors + let mut meta = Vec::with_capacity(partition_count); + for result in results { + match result { + Ok(partition_result) => { + let final_result = partition_result?; + debug!("Query stage partition result: {:?}", final_result); + meta.extend(final_result); + } + Err(e) => { + return Err(BallistaError::General(format!( + "Query stage {} failed: {:?}", + stage_id, e + ))) + } + } + } + + debug!( + "execute_query_stage() stage_id={} produced {:?}", + stage_id, meta + ); + + Ok(meta) +} + +#[cfg(test)] +mod test { + use crate::planner::DistributedPlanner; + use crate::test_utils::datafusion_test_context; + use ballista_core::error::BallistaError; + use ballista_core::execution_plans::UnresolvedShuffleExec; + use ballista_core::serde::protobuf; + use ballista_core::serde::scheduler::ExecutorMeta; + use ballista_core::utils::format_plan; + use datafusion::physical_plan::hash_aggregate::HashAggregateExec; + use datafusion::physical_plan::merge::MergeExec; + use datafusion::physical_plan::projection::ProjectionExec; + use datafusion::physical_plan::sort::SortExec; + use datafusion::physical_plan::ExecutionPlan; + use std::convert::TryInto; + use std::sync::Arc; + use uuid::Uuid; + + macro_rules! downcast_exec { + ($exec: expr, $ty: ty) => { + $exec.as_any().downcast_ref::<$ty>().unwrap() + }; + } + + #[test] + fn test() -> Result<(), BallistaError> { + let mut ctx = datafusion_test_context("testdata")?; + + // simplified form of TPC-H query 1 + let df = ctx.sql( + "select l_returnflag, sum(l_extendedprice * 1) as sum_disc_price + from lineitem + group by l_returnflag + order by l_returnflag", + )?; + + let plan = df.to_logical_plan(); + let plan = ctx.optimize(&plan)?; + let plan = ctx.create_physical_plan(&plan)?; + + let mut planner = DistributedPlanner::try_new(vec![ExecutorMeta { + id: "".to_string(), + host: "".to_string(), + port: 0, + }])?; + let job_uuid = Uuid::new_v4(); + let stages = planner.plan_query_stages(&job_uuid.to_string(), plan)?; + for stage in &stages { + println!("{}", format_plan(stage.as_ref(), 0)?); + } + + /* Expected result: + QueryStageExec: job=f011432e-e424-4016-915d-e3d8b84f6dbd, stage=1 + HashAggregateExec: groupBy=["l_returnflag"], aggrExpr=["SUM(l_extendedprice Multiply Int64(1)) [\"l_extendedprice * CAST(1 AS Float64)\"]"] + CsvExec: testdata/lineitem; partitions=2 + + QueryStageExec: job=f011432e-e424-4016-915d-e3d8b84f6dbd, stage=2 + MergeExec + UnresolvedShuffleExec: stages=[1] + + QueryStageExec: job=f011432e-e424-4016-915d-e3d8b84f6dbd, stage=3 + SortExec { input: ProjectionExec { expr: [(Column { name: "l_returnflag" }, "l_returnflag"), (Column { name: "SUM(l_ext + ProjectionExec { expr: [(Column { name: "l_returnflag" }, "l_returnflag"), (Column { name: "SUM(l_extendedprice Multip + HashAggregateExec: groupBy=["l_returnflag"], aggrExpr=["SUM(l_extendedprice Multiply Int64(1)) [\"l_extendedprice * CAST(1 AS Float64)\"]"] + UnresolvedShuffleExec: stages=[2] + */ + + let sort = stages[2].children()[0].clone(); + let sort = downcast_exec!(sort, SortExec); + + let projection = sort.children()[0].clone(); + println!("{:?}", projection); + let projection = downcast_exec!(projection, ProjectionExec); + + let final_hash = projection.children()[0].clone(); + let final_hash = downcast_exec!(final_hash, HashAggregateExec); + + let unresolved_shuffle = final_hash.children()[0].clone(); + let unresolved_shuffle = downcast_exec!(unresolved_shuffle, UnresolvedShuffleExec); + assert_eq!(unresolved_shuffle.query_stage_ids, vec![2]); + + let merge_exec = stages[1].children()[0].clone(); + let merge_exec = downcast_exec!(merge_exec, MergeExec); + + let unresolved_shuffle = merge_exec.children()[0].clone(); + let unresolved_shuffle = downcast_exec!(unresolved_shuffle, UnresolvedShuffleExec); + assert_eq!(unresolved_shuffle.query_stage_ids, vec![1]); + + let partial_hash = stages[0].children()[0].clone(); + let partial_hash_serde = roundtrip_operator(partial_hash.clone())?; + + let partial_hash = downcast_exec!(partial_hash, HashAggregateExec); + let partial_hash_serde = downcast_exec!(partial_hash_serde, HashAggregateExec); + + assert_eq!( + format!("{:?}", partial_hash), + format!("{:?}", partial_hash_serde) + ); + + Ok(()) + } + + fn roundtrip_operator( + plan: Arc, + ) -> Result, BallistaError> { + let proto: protobuf::PhysicalPlanNode = plan.clone().try_into()?; + let result_exec_plan: Arc = (&proto).try_into()?; + Ok(result_exec_plan) + } +} diff --git a/rust/ballista/rust/scheduler/src/state/etcd.rs b/rust/ballista/rust/scheduler/src/state/etcd.rs new file mode 100644 index 0000000000000..1aaf543e3827a --- /dev/null +++ b/rust/ballista/rust/scheduler/src/state/etcd.rs @@ -0,0 +1,116 @@ +// 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. + +//! Etcd config backend. + +use std::time::Duration; + +use crate::state::ConfigBackendClient; +use ballista_core::error::{ballista_error, Result}; + +use etcd_client::{GetOptions, LockResponse, PutOptions}; +use log::warn; + +use super::Lock; + +/// A [`ConfigBackendClient`] implementation that uses etcd to save cluster configuration. +#[derive(Clone)] +pub struct EtcdClient { + etcd: etcd_client::Client, +} + +impl EtcdClient { + pub fn new(etcd: etcd_client::Client) -> Self { + Self { etcd } + } +} + +#[tonic::async_trait] +impl ConfigBackendClient for EtcdClient { + async fn get(&self, key: &str) -> Result> { + Ok(self + .etcd + .clone() + .get(key, None) + .await + .map_err(|e| ballista_error(&format!("etcd error {:?}", e)))? + .kvs() + .get(0) + .map(|kv| kv.value().to_owned()) + .unwrap_or_default()) + } + + async fn get_from_prefix(&self, prefix: &str) -> Result)>> { + Ok(self + .etcd + .clone() + .get(prefix, Some(GetOptions::new().with_prefix())) + .await + .map_err(|e| ballista_error(&format!("etcd error {:?}", e)))? + .kvs() + .iter() + .map(|kv| (kv.key_str().unwrap().to_owned(), kv.value().to_owned())) + .collect()) + } + + async fn put(&self, key: String, value: Vec, lease_time: Option) -> Result<()> { + let mut etcd = self.etcd.clone(); + let put_options = if let Some(lease_time) = lease_time { + etcd.lease_grant(lease_time.as_secs() as i64, None) + .await + .map(|lease| Some(PutOptions::new().with_lease(lease.id()))) + .map_err(|e| { + warn!("etcd lease grant failed: {:?}", e.to_string()); + ballista_error("etcd lease grant failed") + })? + } else { + None + }; + etcd.put(key.clone(), value.clone(), put_options) + .await + .map_err(|e| { + warn!("etcd put failed: {}", e); + ballista_error("etcd put failed") + }) + .map(|_| ()) + } + + async fn lock(&self) -> Result> { + let mut etcd = self.etcd.clone(); + let lock = etcd + .lock("/ballista_global_lock", None) + .await + .map_err(|e| { + warn!("etcd lock failed: {}", e); + ballista_error("etcd lock failed") + })?; + Ok(Box::new(EtcdLockGuard { etcd, lock })) + } +} + +struct EtcdLockGuard { + etcd: etcd_client::Client, + lock: LockResponse, +} + +// Cannot use Drop because we need this to be async +#[tonic::async_trait] +impl Lock for EtcdLockGuard { + async fn unlock(&mut self) { + self.etcd.unlock(self.lock.key()).await.unwrap(); + } +} diff --git a/rust/ballista/rust/scheduler/src/state/mod.rs b/rust/ballista/rust/scheduler/src/state/mod.rs new file mode 100644 index 0000000000000..bcefc707c04f9 --- /dev/null +++ b/rust/ballista/rust/scheduler/src/state/mod.rs @@ -0,0 +1,792 @@ +// 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. + +use std::{any::type_name, collections::HashMap, convert::TryInto, sync::Arc, time::Duration}; + +use datafusion::physical_plan::ExecutionPlan; +use log::{debug, info}; +use prost::Message; +use tokio::sync::OwnedMutexGuard; + +use ballista_core::serde::protobuf::{ + job_status, task_status, CompletedJob, CompletedTask, ExecutorMetadata, FailedJob, FailedTask, + JobStatus, PhysicalPlanNode, RunningJob, RunningTask, TaskStatus, +}; +use ballista_core::serde::scheduler::PartitionStats; +use ballista_core::{error::BallistaError, serde::scheduler::ExecutorMeta}; +use ballista_core::{ + error::Result, execution_plans::UnresolvedShuffleExec, serde::protobuf::PartitionLocation, +}; + +use super::planner::remove_unresolved_shuffles; + +#[cfg(feature = "etcd")] +mod etcd; +#[cfg(feature = "sled")] +mod standalone; + +#[cfg(feature = "etcd")] +pub use etcd::EtcdClient; +#[cfg(feature = "sled")] +pub use standalone::StandaloneClient; + +const LEASE_TIME: Duration = Duration::from_secs(60); + +/// A trait that contains the necessary methods to save and retrieve the state and configuration of a cluster. +#[tonic::async_trait] +pub trait ConfigBackendClient: Send + Sync { + /// Retrieve the data associated with a specific key. + /// + /// An empty vec is returned if the key does not exist. + async fn get(&self, key: &str) -> Result>; + + /// Retrieve all data associated with a specific key. + async fn get_from_prefix(&self, prefix: &str) -> Result)>>; + + /// Saves the value into the provided key, overriding any previous data that might have been associated to that key. + async fn put(&self, key: String, value: Vec, lease_time: Option) -> Result<()>; + + async fn lock(&self) -> Result>; +} + +#[derive(Clone)] +pub(super) struct SchedulerState { + config_client: Arc, +} + +impl SchedulerState { + pub fn new(config_client: Arc) -> Self { + Self { config_client } + } + + pub async fn get_executors_metadata(&self, namespace: &str) -> Result> { + let mut result = vec![]; + + let entries = self + .config_client + .get_from_prefix(&get_executors_prefix(namespace)) + .await?; + for (_key, entry) in entries { + let meta: ExecutorMetadata = decode_protobuf(&entry)?; + result.push(meta.into()); + } + Ok(result) + } + + pub async fn save_executor_metadata(&self, namespace: &str, meta: ExecutorMeta) -> Result<()> { + let key = get_executor_key(namespace, &meta.id); + let meta: ExecutorMetadata = meta.into(); + let value: Vec = encode_protobuf(&meta)?; + self.config_client.put(key, value, Some(LEASE_TIME)).await + } + + pub async fn save_job_metadata( + &self, + namespace: &str, + job_id: &str, + status: &JobStatus, + ) -> Result<()> { + debug!("Saving job metadata: {:?}", status); + let key = get_job_key(namespace, job_id); + let value = encode_protobuf(status)?; + self.config_client.put(key, value, None).await + } + + pub async fn get_job_metadata(&self, namespace: &str, job_id: &str) -> Result { + let key = get_job_key(namespace, job_id); + let value = &self.config_client.get(&key).await?; + if value.is_empty() { + return Err(BallistaError::General(format!( + "No job metadata found for {}", + key + ))); + } + let value: JobStatus = decode_protobuf(value)?; + Ok(value) + } + + pub async fn save_task_status(&self, namespace: &str, status: &TaskStatus) -> Result<()> { + let partition_id = status.partition_id.as_ref().unwrap(); + let key = get_task_status_key( + namespace, + &partition_id.job_id, + partition_id.stage_id as usize, + partition_id.partition_id as usize, + ); + let value = encode_protobuf(status)?; + self.config_client.put(key, value, None).await + } + + pub async fn _get_task_status( + &self, + namespace: &str, + job_id: &str, + stage_id: usize, + partition_id: usize, + ) -> Result { + let key = get_task_status_key(namespace, job_id, stage_id, partition_id); + let value = &self.config_client.clone().get(&key).await?; + if value.is_empty() { + return Err(BallistaError::General(format!( + "No task status found for {}", + key + ))); + } + let value: TaskStatus = decode_protobuf(value)?; + Ok(value) + } + + // "Unnecessary" lifetime syntax due to https://github.com/rust-lang/rust/issues/63033 + pub async fn save_stage_plan<'a>( + &'a self, + namespace: &'a str, + job_id: &'a str, + stage_id: usize, + plan: Arc, + ) -> Result<()> { + let key = get_stage_plan_key(namespace, job_id, stage_id); + let value = { + let proto: PhysicalPlanNode = plan.try_into()?; + encode_protobuf(&proto)? + }; + self.config_client.clone().put(key, value, None).await + } + + pub async fn get_stage_plan( + &self, + namespace: &str, + job_id: &str, + stage_id: usize, + ) -> Result> { + let key = get_stage_plan_key(namespace, job_id, stage_id); + let value = &self.config_client.get(&key).await?; + if value.is_empty() { + return Err(BallistaError::General(format!( + "No stage plan found for {}", + key + ))); + } + let value: PhysicalPlanNode = decode_protobuf(value)?; + Ok((&value).try_into()?) + } + + pub async fn assign_next_schedulable_task( + &self, + namespace: &str, + executor_id: &str, + ) -> Result)>> { + let kvs: HashMap> = self + .config_client + .get_from_prefix(&get_task_prefix(namespace)) + .await? + .into_iter() + .collect(); + let executors = self.get_executors_metadata(namespace).await?; + 'tasks: for (_key, value) in kvs.iter() { + let mut status: TaskStatus = decode_protobuf(&value)?; + if status.status.is_none() { + let partition = status.partition_id.as_ref().unwrap(); + let plan = self + .get_stage_plan(namespace, &partition.job_id, partition.stage_id as usize) + .await?; + + // Let's try to resolve any unresolved shuffles we find + let unresolved_shuffles = find_unresolved_shuffles(&plan)?; + let mut partition_locations: HashMap< + usize, + Vec, + > = HashMap::new(); + for unresolved_shuffle in unresolved_shuffles { + for stage_id in unresolved_shuffle.query_stage_ids { + for partition_id in 0..unresolved_shuffle.partition_count { + let referenced_task = kvs + .get(&get_task_status_key( + namespace, + &partition.job_id, + stage_id, + partition_id, + )) + .unwrap(); + let referenced_task: TaskStatus = decode_protobuf(referenced_task)?; + if let Some(task_status::Status::Completed(CompletedTask { + executor_id, + })) = referenced_task.status + { + let empty = vec![]; + let locations = + partition_locations.entry(stage_id).or_insert(empty); + locations.push( + ballista_core::serde::scheduler::PartitionLocation { + partition_id: + ballista_core::serde::scheduler::PartitionId { + job_id: partition.job_id.clone(), + stage_id, + partition_id, + }, + executor_meta: executors + .iter() + .find(|exec| exec.id == executor_id) + .unwrap() + .clone(), + partition_stats: PartitionStats::default(), + }, + ); + } else { + continue 'tasks; + } + } + } + } + let plan = remove_unresolved_shuffles(plan.as_ref(), &partition_locations)?; + + // If we get here, there are no more unresolved shuffled and the task can be run + status.status = Some(task_status::Status::Running(RunningTask { + executor_id: executor_id.to_owned(), + })); + self.save_task_status(namespace, &status).await?; + return Ok(Some((status, plan))); + } + } + Ok(None) + } + + // Global lock for the state. We should get rid of this to be able to scale. + pub async fn lock(&self) -> Result> { + self.config_client.lock().await + } + + pub async fn synchronize_job_status(&self, namespace: &str) -> Result<()> { + let kvs = self + .config_client + .get_from_prefix(&get_job_prefix(namespace)) + .await?; + let executors: HashMap = self + .get_executors_metadata(namespace) + .await? + .into_iter() + .map(|meta| (meta.id.to_string(), meta)) + .collect(); + for (key, value) in kvs { + let job_id = extract_job_id_from_key(&key)?; + let status: JobStatus = decode_protobuf(&value)?; + let new_status = self + .get_job_status_from_tasks(namespace, job_id, &executors) + .await?; + if let Some(new_status) = new_status { + if status != new_status { + info!( + "Changing status for job {} to {:?}", + job_id, new_status.status + ); + debug!("Old status: {:?}", status); + debug!("New status: {:?}", new_status); + self.save_job_metadata(namespace, job_id, &new_status) + .await?; + } + } + } + Ok(()) + } + + async fn get_job_status_from_tasks( + &self, + namespace: &str, + job_id: &str, + executors: &HashMap, + ) -> Result> { + let statuses = self + .config_client + .get_from_prefix(&get_task_prefix_for_job(namespace, job_id)) + .await? + .into_iter() + .map(|(_k, v)| decode_protobuf::(&v)) + .collect::>>()?; + if statuses.is_empty() { + return Ok(None); + } + + // Check for job completion + let mut job_status = statuses + .iter() + .map(|status| match &status.status { + Some(task_status::Status::Completed(CompletedTask { executor_id })) => { + Ok((status, executor_id)) + } + _ => Err(BallistaError::General("Task not completed".to_string())), + }) + .collect::>>() + .ok() + .map(|info| { + let partition_location = info + .into_iter() + .map(|(status, execution_id)| PartitionLocation { + partition_id: status.partition_id.to_owned(), + executor_meta: executors.get(execution_id).map(|e| e.clone().into()), + partition_stats: None, + }) + .collect(); + job_status::Status::Completed(CompletedJob { partition_location }) + }); + + if job_status.is_none() { + // Update other statuses + for status in statuses { + match status.status { + Some(task_status::Status::Failed(FailedTask { error })) => { + job_status = Some(job_status::Status::Failed(FailedJob { error })); + break; + } + Some(task_status::Status::Running(_)) if job_status == None => { + job_status = Some(job_status::Status::Running(RunningJob {})); + } + _ => (), + } + } + } + Ok(job_status.map(|status| JobStatus { + status: Some(status), + })) + } +} + +#[tonic::async_trait] +pub trait Lock: Send + Sync { + async fn unlock(&mut self); +} + +#[tonic::async_trait] +impl Lock for OwnedMutexGuard { + async fn unlock(&mut self) {} +} + +/// Returns the the unresolved shuffles in the execution plan +fn find_unresolved_shuffles(plan: &Arc) -> Result> { + if let Some(unresolved_shuffle) = plan.as_any().downcast_ref::() { + Ok(vec![unresolved_shuffle.clone()]) + } else { + Ok(plan + .children() + .iter() + .map(|child| find_unresolved_shuffles(child)) + .collect::>>()? + .into_iter() + .flatten() + .collect()) + } +} + +fn get_executors_prefix(namespace: &str) -> String { + format!("/ballista/{}/executors", namespace) +} + +fn get_executor_key(namespace: &str, id: &str) -> String { + format!("{}/{}", get_executors_prefix(namespace), id) +} + +fn get_job_prefix(namespace: &str) -> String { + format!("/ballista/{}/jobs", namespace) +} + +fn extract_job_id_from_key(job_key: &str) -> Result<&str> { + job_key + .split('/') + .nth(4) + .ok_or_else(|| BallistaError::Internal(format!("Unexpected job key: {}", job_key))) +} + +fn get_job_key(namespace: &str, id: &str) -> String { + format!("{}/{}", get_job_prefix(namespace), id) +} + +fn get_task_prefix(namespace: &str) -> String { + format!("/ballista/{}/tasks", namespace) +} + +fn get_task_prefix_for_job(namespace: &str, job_id: &str) -> String { + format!("{}/{}", get_task_prefix(namespace), job_id) +} + +fn get_task_status_key( + namespace: &str, + job_id: &str, + stage_id: usize, + partition_id: usize, +) -> String { + format!( + "{}/{}/{}", + get_task_prefix_for_job(namespace, job_id), + stage_id, + partition_id, + ) +} + +fn get_stage_plan_key(namespace: &str, job_id: &str, stage_id: usize) -> String { + format!("/ballista/{}/stages/{}/{}", namespace, job_id, stage_id,) +} + +fn decode_protobuf(bytes: &[u8]) -> Result { + T::decode(bytes).map_err(|e| { + BallistaError::Internal(format!("Could not deserialize {}: {}", type_name::(), e)) + }) +} + +fn encode_protobuf(msg: &T) -> Result> { + let mut value: Vec = Vec::with_capacity(msg.encoded_len()); + msg.encode(&mut value).map_err(|e| { + BallistaError::Internal(format!("Could not serialize {}: {}", type_name::(), e)) + })?; + Ok(value) +} + +#[cfg(all(test, feature = "sled"))] +mod test { + use std::sync::Arc; + + use ballista_core::serde::protobuf::{ + job_status, task_status, CompletedTask, FailedTask, JobStatus, PartitionId, QueuedJob, + RunningJob, RunningTask, TaskStatus, + }; + use ballista_core::{error::BallistaError, serde::scheduler::ExecutorMeta}; + + use super::{SchedulerState, StandaloneClient}; + + #[tokio::test] + async fn executor_metadata() -> Result<(), BallistaError> { + let state = SchedulerState::new(Arc::new(StandaloneClient::try_new_temporary()?)); + let meta = ExecutorMeta { + id: "123".to_owned(), + host: "localhost".to_owned(), + port: 123, + }; + state.save_executor_metadata("test", meta.clone()).await?; + let result = state.get_executors_metadata("test").await?; + assert_eq!(vec![meta], result); + Ok(()) + } + + #[tokio::test] + async fn executor_metadata_empty() -> Result<(), BallistaError> { + let state = SchedulerState::new(Arc::new(StandaloneClient::try_new_temporary()?)); + let meta = ExecutorMeta { + id: "123".to_owned(), + host: "localhost".to_owned(), + port: 123, + }; + state.save_executor_metadata("test", meta.clone()).await?; + let result = state.get_executors_metadata("test2").await?; + assert!(result.is_empty()); + Ok(()) + } + + #[tokio::test] + async fn job_metadata() -> Result<(), BallistaError> { + let state = SchedulerState::new(Arc::new(StandaloneClient::try_new_temporary()?)); + let meta = JobStatus { + status: Some(job_status::Status::Queued(QueuedJob {})), + }; + state.save_job_metadata("test", "job", &meta).await?; + let result = state.get_job_metadata("test", "job").await?; + assert!(result.status.is_some()); + match result.status.unwrap() { + job_status::Status::Queued(_) => (), + _ => panic!("Unexpected status"), + } + Ok(()) + } + + #[tokio::test] + async fn job_metadata_non_existant() -> Result<(), BallistaError> { + let state = SchedulerState::new(Arc::new(StandaloneClient::try_new_temporary()?)); + let meta = JobStatus { + status: Some(job_status::Status::Queued(QueuedJob {})), + }; + state.save_job_metadata("test", "job", &meta).await?; + let result = state.get_job_metadata("test2", "job2").await; + assert!(result.is_err()); + Ok(()) + } + + #[tokio::test] + async fn task_status() -> Result<(), BallistaError> { + let state = SchedulerState::new(Arc::new(StandaloneClient::try_new_temporary()?)); + let meta = TaskStatus { + status: Some(task_status::Status::Failed(FailedTask { + error: "error".to_owned(), + })), + partition_id: Some(PartitionId { + job_id: "job".to_owned(), + stage_id: 1, + partition_id: 2, + }), + }; + state.save_task_status("test", &meta).await?; + let result = state._get_task_status("test", "job", 1, 2).await?; + assert!(result.status.is_some()); + match result.status.unwrap() { + task_status::Status::Failed(_) => (), + _ => panic!("Unexpected status"), + } + Ok(()) + } + + #[tokio::test] + async fn task_status_non_existant() -> Result<(), BallistaError> { + let state = SchedulerState::new(Arc::new(StandaloneClient::try_new_temporary()?)); + let meta = TaskStatus { + status: Some(task_status::Status::Failed(FailedTask { + error: "error".to_owned(), + })), + partition_id: Some(PartitionId { + job_id: "job".to_owned(), + stage_id: 1, + partition_id: 2, + }), + }; + state.save_task_status("test", &meta).await?; + let result = state._get_task_status("test", "job", 25, 2).await; + assert!(result.is_err()); + Ok(()) + } + + #[tokio::test] + async fn task_synchronize_job_status_queued() -> Result<(), BallistaError> { + let state = SchedulerState::new(Arc::new(StandaloneClient::try_new_temporary()?)); + let namespace = "default"; + let job_id = "job"; + let job_status = JobStatus { + status: Some(job_status::Status::Queued(QueuedJob {})), + }; + state + .save_job_metadata(namespace, job_id, &job_status) + .await?; + state.synchronize_job_status(namespace).await?; + let result = state.get_job_metadata(namespace, job_id).await?; + assert_eq!(result, job_status); + Ok(()) + } + + #[tokio::test] + async fn task_synchronize_job_status_running() -> Result<(), BallistaError> { + let state = SchedulerState::new(Arc::new(StandaloneClient::try_new_temporary()?)); + let namespace = "default"; + let job_id = "job"; + let job_status = JobStatus { + status: Some(job_status::Status::Running(RunningJob {})), + }; + state + .save_job_metadata(namespace, job_id, &job_status) + .await?; + let meta = TaskStatus { + status: Some(task_status::Status::Completed(CompletedTask { + executor_id: "".to_owned(), + })), + partition_id: Some(PartitionId { + job_id: job_id.to_owned(), + stage_id: 0, + partition_id: 0, + }), + }; + state.save_task_status(namespace, &meta).await?; + let meta = TaskStatus { + status: Some(task_status::Status::Running(RunningTask { + executor_id: "".to_owned(), + })), + partition_id: Some(PartitionId { + job_id: job_id.to_owned(), + stage_id: 0, + partition_id: 1, + }), + }; + state.save_task_status(namespace, &meta).await?; + state.synchronize_job_status(namespace).await?; + let result = state.get_job_metadata(namespace, job_id).await?; + assert_eq!(result, job_status); + Ok(()) + } + + #[tokio::test] + async fn task_synchronize_job_status_running2() -> Result<(), BallistaError> { + let state = SchedulerState::new(Arc::new(StandaloneClient::try_new_temporary()?)); + let namespace = "default"; + let job_id = "job"; + let job_status = JobStatus { + status: Some(job_status::Status::Running(RunningJob {})), + }; + state + .save_job_metadata(namespace, job_id, &job_status) + .await?; + let meta = TaskStatus { + status: Some(task_status::Status::Completed(CompletedTask { + executor_id: "".to_owned(), + })), + partition_id: Some(PartitionId { + job_id: job_id.to_owned(), + stage_id: 0, + partition_id: 0, + }), + }; + state.save_task_status(namespace, &meta).await?; + let meta = TaskStatus { + status: None, + partition_id: Some(PartitionId { + job_id: job_id.to_owned(), + stage_id: 0, + partition_id: 1, + }), + }; + state.save_task_status(namespace, &meta).await?; + state.synchronize_job_status(namespace).await?; + let result = state.get_job_metadata(namespace, job_id).await?; + assert_eq!(result, job_status); + Ok(()) + } + + #[tokio::test] + async fn task_synchronize_job_status_completed() -> Result<(), BallistaError> { + let state = SchedulerState::new(Arc::new(StandaloneClient::try_new_temporary()?)); + let namespace = "default"; + let job_id = "job"; + let job_status = JobStatus { + status: Some(job_status::Status::Running(RunningJob {})), + }; + state + .save_job_metadata(namespace, job_id, &job_status) + .await?; + let meta = TaskStatus { + status: Some(task_status::Status::Completed(CompletedTask { + executor_id: "".to_owned(), + })), + partition_id: Some(PartitionId { + job_id: job_id.to_owned(), + stage_id: 0, + partition_id: 0, + }), + }; + state.save_task_status(namespace, &meta).await?; + let meta = TaskStatus { + status: Some(task_status::Status::Completed(CompletedTask { + executor_id: "".to_owned(), + })), + partition_id: Some(PartitionId { + job_id: job_id.to_owned(), + stage_id: 0, + partition_id: 1, + }), + }; + state.save_task_status(namespace, &meta).await?; + state.synchronize_job_status(namespace).await?; + let result = state.get_job_metadata(namespace, job_id).await?; + match result.status.unwrap() { + job_status::Status::Completed(_) => (), + status => panic!("Received status: {:?}", status), + } + Ok(()) + } + + #[tokio::test] + async fn task_synchronize_job_status_completed2() -> Result<(), BallistaError> { + let state = SchedulerState::new(Arc::new(StandaloneClient::try_new_temporary()?)); + let namespace = "default"; + let job_id = "job"; + let job_status = JobStatus { + status: Some(job_status::Status::Queued(QueuedJob {})), + }; + state + .save_job_metadata(namespace, job_id, &job_status) + .await?; + let meta = TaskStatus { + status: Some(task_status::Status::Completed(CompletedTask { + executor_id: "".to_owned(), + })), + partition_id: Some(PartitionId { + job_id: job_id.to_owned(), + stage_id: 0, + partition_id: 0, + }), + }; + state.save_task_status(namespace, &meta).await?; + let meta = TaskStatus { + status: Some(task_status::Status::Completed(CompletedTask { + executor_id: "".to_owned(), + })), + partition_id: Some(PartitionId { + job_id: job_id.to_owned(), + stage_id: 0, + partition_id: 1, + }), + }; + state.save_task_status(namespace, &meta).await?; + state.synchronize_job_status(namespace).await?; + let result = state.get_job_metadata(namespace, job_id).await?; + match result.status.unwrap() { + job_status::Status::Completed(_) => (), + status => panic!("Received status: {:?}", status), + } + Ok(()) + } + + #[tokio::test] + async fn task_synchronize_job_status_failed() -> Result<(), BallistaError> { + let state = SchedulerState::new(Arc::new(StandaloneClient::try_new_temporary()?)); + let namespace = "default"; + let job_id = "job"; + let job_status = JobStatus { + status: Some(job_status::Status::Running(RunningJob {})), + }; + state + .save_job_metadata(namespace, job_id, &job_status) + .await?; + let meta = TaskStatus { + status: Some(task_status::Status::Completed(CompletedTask { + executor_id: "".to_owned(), + })), + partition_id: Some(PartitionId { + job_id: job_id.to_owned(), + stage_id: 0, + partition_id: 0, + }), + }; + state.save_task_status(namespace, &meta).await?; + let meta = TaskStatus { + status: Some(task_status::Status::Failed(FailedTask { + error: "".to_owned(), + })), + partition_id: Some(PartitionId { + job_id: job_id.to_owned(), + stage_id: 0, + partition_id: 1, + }), + }; + state.save_task_status(namespace, &meta).await?; + let meta = TaskStatus { + status: None, + partition_id: Some(PartitionId { + job_id: job_id.to_owned(), + stage_id: 0, + partition_id: 2, + }), + }; + state.save_task_status(namespace, &meta).await?; + state.synchronize_job_status(namespace).await?; + let result = state.get_job_metadata(namespace, job_id).await?; + match result.status.unwrap() { + job_status::Status::Failed(_) => (), + status => panic!("Received status: {:?}", status), + } + Ok(()) + } +} diff --git a/rust/ballista/rust/scheduler/src/state/standalone.rs b/rust/ballista/rust/scheduler/src/state/standalone.rs new file mode 100644 index 0000000000000..0c7a5ae8884a4 --- /dev/null +++ b/rust/ballista/rust/scheduler/src/state/standalone.rs @@ -0,0 +1,156 @@ +// 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. + +use std::{sync::Arc, time::Duration}; + +use crate::state::ConfigBackendClient; +use ballista_core::error::{ballista_error, BallistaError, Result}; + +use log::warn; +use tokio::sync::Mutex; + +use super::Lock; + +/// A [`ConfigBackendClient`] implementation that uses file-based storage to save cluster configuration. +#[derive(Clone)] +pub struct StandaloneClient { + db: sled::Db, + lock: Arc>, +} + +impl StandaloneClient { + /// Creates a StandaloneClient that saves data to the specified file. + pub fn try_new>(path: P) -> Result { + Ok(Self { + db: sled::open(path).map_err(sled_to_ballista_error)?, + lock: Arc::new(Mutex::new(())), + }) + } + + /// Creates a StandaloneClient that saves data to a temp file. + pub fn try_new_temporary() -> Result { + Ok(Self { + db: sled::Config::new() + .temporary(true) + .open() + .map_err(sled_to_ballista_error)?, + lock: Arc::new(Mutex::new(())), + }) + } +} + +fn sled_to_ballista_error(e: sled::Error) -> BallistaError { + match e { + sled::Error::Io(io) => BallistaError::IoError(io), + _ => BallistaError::General(format!("{}", e)), + } +} + +#[tonic::async_trait] +impl ConfigBackendClient for StandaloneClient { + async fn get(&self, key: &str) -> Result> { + Ok(self + .db + .get(key) + .map_err(|e| ballista_error(&format!("sled error {:?}", e)))? + .map(|v| v.to_vec()) + .unwrap_or_default()) + } + + async fn get_from_prefix(&self, prefix: &str) -> Result)>> { + Ok(self + .db + .scan_prefix(prefix) + .map(|v| { + v.map(|(key, value)| { + ( + std::str::from_utf8(&key).unwrap().to_owned(), + value.to_vec(), + ) + }) + }) + .collect::, _>>() + .map_err(|e| ballista_error(&format!("sled error {:?}", e)))?) + } + + // TODO: support lease_time. See https://github.com/spacejam/sled/issues/1119 for how to approach this + async fn put(&self, key: String, value: Vec, _lease_time: Option) -> Result<()> { + self.db + .insert(key, value) + .map_err(|e| { + warn!("sled insert failed: {}", e); + ballista_error("sled insert failed") + }) + .map(|_| ()) + } + + async fn lock(&self) -> Result> { + Ok(Box::new(self.lock.clone().lock_owned().await)) + } +} + +#[cfg(test)] +mod tests { + use crate::state::ConfigBackendClient; + + use super::StandaloneClient; + use std::result::Result; + + fn create_instance() -> Result> { + Ok(StandaloneClient::try_new_temporary()?) + } + + #[tokio::test] + async fn put_read() -> Result<(), Box> { + let client = create_instance()?; + let key = "key"; + let value = "value".as_bytes(); + client.put(key.to_owned(), value.to_vec(), None).await?; + assert_eq!(client.get(key).await?, value); + Ok(()) + } + + #[tokio::test] + async fn read_empty() -> Result<(), Box> { + let client = create_instance()?; + let key = "key"; + let empty: &[u8] = &[]; + assert_eq!(client.get(key).await?, empty); + Ok(()) + } + + #[tokio::test] + async fn read_prefix() -> Result<(), Box> { + let client = create_instance()?; + let key = "key"; + let value = "value".as_bytes(); + client + .put(format!("{}/1", key), value.to_vec(), None) + .await?; + client + .put(format!("{}/2", key), value.to_vec(), None) + .await?; + assert_eq!( + client.get_from_prefix(key).await?, + vec![ + ("key/1".to_owned(), value.to_vec()), + ("key/2".to_owned(), value.to_vec()) + ] + ); + Ok(()) + } +} diff --git a/rust/ballista/rust/scheduler/src/test_utils.rs b/rust/ballista/rust/scheduler/src/test_utils.rs new file mode 100644 index 0000000000000..9439740477777 --- /dev/null +++ b/rust/ballista/rust/scheduler/src/test_utils.rs @@ -0,0 +1,136 @@ +// 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. + +use ballista_core::error::Result; + +use arrow::datatypes::{DataType, Field, Schema}; +use datafusion::execution::context::ExecutionContext; +use datafusion::physical_plan::csv::CsvReadOptions; + +pub const TPCH_TABLES: &[&str] = &[ + "part", "supplier", "partsupp", "customer", "orders", "lineitem", "nation", "region", +]; + +pub fn datafusion_test_context(path: &str) -> Result { + let mut ctx = ExecutionContext::new(); + for table in TPCH_TABLES { + let schema = get_tpch_schema(table); + let options = CsvReadOptions::new() + .schema(&schema) + .delimiter(b'|') + .has_header(false) + .file_extension(".tbl"); + let dir = format!("{}/{}", path, table); + ctx.register_csv(table, &dir, options)?; + } + Ok(ctx) +} + +pub fn get_tpch_schema(table: &str) -> Schema { + // note that the schema intentionally uses signed integers so that any generated Parquet + // files can also be used to benchmark tools that only support signed integers, such as + // Apache Spark + + match table { + "part" => Schema::new(vec![ + Field::new("p_partkey", DataType::Int32, false), + Field::new("p_name", DataType::Utf8, false), + Field::new("p_mfgr", DataType::Utf8, false), + Field::new("p_brand", DataType::Utf8, false), + Field::new("p_type", DataType::Utf8, false), + Field::new("p_size", DataType::Int32, false), + Field::new("p_container", DataType::Utf8, false), + Field::new("p_retailprice", DataType::Float64, false), + Field::new("p_comment", DataType::Utf8, false), + ]), + + "supplier" => Schema::new(vec![ + Field::new("s_suppkey", DataType::Int32, false), + Field::new("s_name", DataType::Utf8, false), + Field::new("s_address", DataType::Utf8, false), + Field::new("s_nationkey", DataType::Int32, false), + Field::new("s_phone", DataType::Utf8, false), + Field::new("s_acctbal", DataType::Float64, false), + Field::new("s_comment", DataType::Utf8, false), + ]), + + "partsupp" => Schema::new(vec![ + Field::new("ps_partkey", DataType::Int32, false), + Field::new("ps_suppkey", DataType::Int32, false), + Field::new("ps_availqty", DataType::Int32, false), + Field::new("ps_supplycost", DataType::Float64, false), + Field::new("ps_comment", DataType::Utf8, false), + ]), + + "customer" => Schema::new(vec![ + Field::new("c_custkey", DataType::Int32, false), + Field::new("c_name", DataType::Utf8, false), + Field::new("c_address", DataType::Utf8, false), + Field::new("c_nationkey", DataType::Int32, false), + Field::new("c_phone", DataType::Utf8, false), + Field::new("c_acctbal", DataType::Float64, false), + Field::new("c_mktsegment", DataType::Utf8, false), + Field::new("c_comment", DataType::Utf8, false), + ]), + + "orders" => Schema::new(vec![ + Field::new("o_orderkey", DataType::Int32, false), + Field::new("o_custkey", DataType::Int32, false), + Field::new("o_orderstatus", DataType::Utf8, false), + Field::new("o_totalprice", DataType::Float64, false), + Field::new("o_orderdate", DataType::Date32, false), + Field::new("o_orderpriority", DataType::Utf8, false), + Field::new("o_clerk", DataType::Utf8, false), + Field::new("o_shippriority", DataType::Int32, false), + Field::new("o_comment", DataType::Utf8, false), + ]), + + "lineitem" => Schema::new(vec![ + Field::new("l_orderkey", DataType::Int32, false), + Field::new("l_partkey", DataType::Int32, false), + Field::new("l_suppkey", DataType::Int32, false), + Field::new("l_linenumber", DataType::Int32, false), + Field::new("l_quantity", DataType::Float64, false), + Field::new("l_extendedprice", DataType::Float64, false), + Field::new("l_discount", DataType::Float64, false), + Field::new("l_tax", DataType::Float64, false), + Field::new("l_returnflag", DataType::Utf8, false), + Field::new("l_linestatus", DataType::Utf8, false), + Field::new("l_shipdate", DataType::Date32, false), + Field::new("l_commitdate", DataType::Date32, false), + Field::new("l_receiptdate", DataType::Date32, false), + Field::new("l_shipinstruct", DataType::Utf8, false), + Field::new("l_shipmode", DataType::Utf8, false), + Field::new("l_comment", DataType::Utf8, false), + ]), + + "nation" => Schema::new(vec![ + Field::new("n_nationkey", DataType::Int32, false), + Field::new("n_name", DataType::Utf8, false), + Field::new("n_regionkey", DataType::Int32, false), + Field::new("n_comment", DataType::Utf8, false), + ]), + + "region" => Schema::new(vec![ + Field::new("r_regionkey", DataType::Int32, false), + Field::new("r_name", DataType::Utf8, false), + Field::new("r_comment", DataType::Utf8, false), + ]), + + _ => unimplemented!(), + } +} diff --git a/rust/ballista/rust/scheduler/testdata/customer/customer.tbl b/rust/ballista/rust/scheduler/testdata/customer/customer.tbl new file mode 100644 index 0000000000000..afa5a739ab32d --- /dev/null +++ b/rust/ballista/rust/scheduler/testdata/customer/customer.tbl @@ -0,0 +1,10 @@ +1|Customer#000000001|IVhzIApeRb ot,c,E|15|25-989-741-2988|711.56|BUILDING|to the even, regular platelets. regular, ironic epitaphs nag e| +2|Customer#000000002|XSTf4,NCwDVaWNe6tEgvwfmRchLXak|13|23-768-687-3665|121.65|AUTOMOBILE|l accounts. blithely ironic theodolites integrate boldly: caref| +3|Customer#000000003|MG9kdTD2WBHm|1|11-719-748-3364|7498.12|AUTOMOBILE| deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov| +4|Customer#000000004|XxVSJsLAGtn|4|14-128-190-5944|2866.83|MACHINERY| requests. final, regular ideas sleep final accou| +5|Customer#000000005|KvpyuHCplrB84WgAiGV6sYpZq7Tj|3|13-750-942-6364|794.47|HOUSEHOLD|n accounts will have to unwind. foxes cajole accor| +6|Customer#000000006|sKZz0CsnMD7mp4Xd0YrBvx,LREYKUWAh yVn|20|30-114-968-4951|7638.57|AUTOMOBILE|tions. even deposits boost according to the slyly bold packages. final accounts cajole requests. furious| +7|Customer#000000007|TcGe5gaZNgVePxU5kRrvXBfkasDTea|18|28-190-982-9759|9561.95|AUTOMOBILE|ainst the ironic, express theodolites. express, even pinto beans among the exp| +8|Customer#000000008|I0B10bB0AymmC, 0PrRYBCP1yGJ8xcBPmWhl5|17|27-147-574-9335|6819.74|BUILDING|among the slyly regular theodolites kindle blithely courts. carefully even theodolites haggle slyly along the ide| +9|Customer#000000009|xKiAFTjUsCuxfeleNqefumTrjS|8|18-338-906-3675|8324.07|FURNITURE|r theodolites according to the requests wake thinly excuses: pending requests haggle furiousl| +10|Customer#000000010|6LrEaV6KR6PLVcgl2ArL Q3rqzLzcT1 v2|5|15-741-346-9870|2753.54|HOUSEHOLD|es regular deposits haggle. fur| diff --git a/rust/ballista/rust/scheduler/testdata/lineitem/partition0.tbl b/rust/ballista/rust/scheduler/testdata/lineitem/partition0.tbl new file mode 100644 index 0000000000000..b7424c2138b78 --- /dev/null +++ b/rust/ballista/rust/scheduler/testdata/lineitem/partition0.tbl @@ -0,0 +1,10 @@ +1|155190|7706|1|17|21168.23|0.04|0.02|N|O|1996-03-13|1996-02-12|1996-03-22|DELIVER IN PERSON|TRUCK|egular courts above the| +1|67310|7311|2|36|45983.16|0.09|0.06|N|O|1996-04-12|1996-02-28|1996-04-20|TAKE BACK RETURN|MAIL|ly final dependencies: slyly bold | +1|63700|3701|3|8|13309.60|0.10|0.02|N|O|1996-01-29|1996-03-05|1996-01-31|TAKE BACK RETURN|REG AIR|riously. regular, express dep| +1|2132|4633|4|28|28955.64|0.09|0.06|N|O|1996-04-21|1996-03-30|1996-05-16|NONE|AIR|lites. fluffily even de| +1|24027|1534|5|24|22824.48|0.10|0.04|N|O|1996-03-30|1996-03-14|1996-04-01|NONE|FOB| pending foxes. slyly re| +1|15635|638|6|32|49620.16|0.07|0.02|N|O|1996-01-30|1996-02-07|1996-02-03|DELIVER IN PERSON|MAIL|arefully slyly ex| +2|106170|1191|1|38|44694.46|0.00|0.05|N|O|1997-01-28|1997-01-14|1997-02-02|TAKE BACK RETURN|RAIL|ven requests. deposits breach a| +3|4297|1798|1|45|54058.05|0.06|0.00|R|F|1994-02-02|1994-01-04|1994-02-23|NONE|AIR|ongside of the furiously brave acco| +3|19036|6540|2|49|46796.47|0.10|0.00|R|F|1993-11-09|1993-12-20|1993-11-24|TAKE BACK RETURN|RAIL| unusual accounts. eve| +3|128449|3474|3|27|39890.88|0.06|0.07|A|F|1994-01-16|1993-11-22|1994-01-23|DELIVER IN PERSON|SHIP|nal foxes wake. | diff --git a/rust/ballista/rust/scheduler/testdata/lineitem/partition1.tbl b/rust/ballista/rust/scheduler/testdata/lineitem/partition1.tbl new file mode 100644 index 0000000000000..b7424c2138b78 --- /dev/null +++ b/rust/ballista/rust/scheduler/testdata/lineitem/partition1.tbl @@ -0,0 +1,10 @@ +1|155190|7706|1|17|21168.23|0.04|0.02|N|O|1996-03-13|1996-02-12|1996-03-22|DELIVER IN PERSON|TRUCK|egular courts above the| +1|67310|7311|2|36|45983.16|0.09|0.06|N|O|1996-04-12|1996-02-28|1996-04-20|TAKE BACK RETURN|MAIL|ly final dependencies: slyly bold | +1|63700|3701|3|8|13309.60|0.10|0.02|N|O|1996-01-29|1996-03-05|1996-01-31|TAKE BACK RETURN|REG AIR|riously. regular, express dep| +1|2132|4633|4|28|28955.64|0.09|0.06|N|O|1996-04-21|1996-03-30|1996-05-16|NONE|AIR|lites. fluffily even de| +1|24027|1534|5|24|22824.48|0.10|0.04|N|O|1996-03-30|1996-03-14|1996-04-01|NONE|FOB| pending foxes. slyly re| +1|15635|638|6|32|49620.16|0.07|0.02|N|O|1996-01-30|1996-02-07|1996-02-03|DELIVER IN PERSON|MAIL|arefully slyly ex| +2|106170|1191|1|38|44694.46|0.00|0.05|N|O|1997-01-28|1997-01-14|1997-02-02|TAKE BACK RETURN|RAIL|ven requests. deposits breach a| +3|4297|1798|1|45|54058.05|0.06|0.00|R|F|1994-02-02|1994-01-04|1994-02-23|NONE|AIR|ongside of the furiously brave acco| +3|19036|6540|2|49|46796.47|0.10|0.00|R|F|1993-11-09|1993-12-20|1993-11-24|TAKE BACK RETURN|RAIL| unusual accounts. eve| +3|128449|3474|3|27|39890.88|0.06|0.07|A|F|1994-01-16|1993-11-22|1994-01-23|DELIVER IN PERSON|SHIP|nal foxes wake. | diff --git a/rust/ballista/rust/scheduler/testdata/nation/nation.tbl b/rust/ballista/rust/scheduler/testdata/nation/nation.tbl new file mode 100644 index 0000000000000..c31ad6be0faca --- /dev/null +++ b/rust/ballista/rust/scheduler/testdata/nation/nation.tbl @@ -0,0 +1,10 @@ +0|ALGERIA|0| haggle. carefully final deposits detect slyly agai| +1|ARGENTINA|1|al foxes promise slyly according to the regular accounts. bold requests alon| +2|BRAZIL|1|y alongside of the pending deposits. carefully special packages are about the ironic forges. slyly special | +3|CANADA|1|eas hang ironic, silent packages. slyly regular packages are furiously over the tithes. fluffily bold| +4|EGYPT|4|y above the carefully unusual theodolites. final dugouts are quickly across the furiously regular d| +5|ETHIOPIA|0|ven packages wake quickly. regu| +6|FRANCE|3|refully final requests. regular, ironi| +7|GERMANY|3|l platelets. regular accounts x-ray: unusual, regular acco| +8|INDIA|2|ss excuses cajole slyly across the packages. deposits print aroun| +9|INDONESIA|2| slyly express asymptotes. regular deposits haggle slyly. carefully ironic hockey players sleep blithely. carefull| diff --git a/rust/ballista/rust/scheduler/testdata/orders/orders.tbl b/rust/ballista/rust/scheduler/testdata/orders/orders.tbl new file mode 100644 index 0000000000000..f5fa65b09a7ad --- /dev/null +++ b/rust/ballista/rust/scheduler/testdata/orders/orders.tbl @@ -0,0 +1,10 @@ +1|36901|O|173665.47|1996-01-02|5-LOW|Clerk#000000951|0|nstructions sleep furiously among | +2|78002|O|46929.18|1996-12-01|1-URGENT|Clerk#000000880|0| foxes. pending accounts at the pending, silent asymptot| +3|123314|F|193846.25|1993-10-14|5-LOW|Clerk#000000955|0|sly final accounts boost. carefully regular ideas cajole carefully. depos| +4|136777|O|32151.78|1995-10-11|5-LOW|Clerk#000000124|0|sits. slyly regular warthogs cajole. regular, regular theodolites acro| +5|44485|F|144659.20|1994-07-30|5-LOW|Clerk#000000925|0|quickly. bold deposits sleep slyly. packages use slyly| +6|55624|F|58749.59|1992-02-21|4-NOT SPECIFIED|Clerk#000000058|0|ggle. special, final requests are against the furiously specia| +7|39136|O|252004.18|1996-01-10|2-HIGH|Clerk#000000470|0|ly special requests | +32|130057|O|208660.75|1995-07-16|2-HIGH|Clerk#000000616|0|ise blithely bold, regular requests. quickly unusual dep| +33|66958|F|163243.98|1993-10-27|3-MEDIUM|Clerk#000000409|0|uriously. furiously final request| +34|61001|O|58949.67|1998-07-21|3-MEDIUM|Clerk#000000223|0|ly final packages. fluffily final deposits wake blithely ideas. spe| diff --git a/rust/ballista/rust/scheduler/testdata/part/part.tbl b/rust/ballista/rust/scheduler/testdata/part/part.tbl new file mode 100644 index 0000000000000..0c6f0e2f3e107 --- /dev/null +++ b/rust/ballista/rust/scheduler/testdata/part/part.tbl @@ -0,0 +1,10 @@ +1|goldenrod lavender spring chocolate lace|Manufacturer#1|Brand#13|PROMO BURNISHED COPPER|7|JUMBO PKG|901.00|ly. slyly ironi| +2|blush thistle blue yellow saddle|Manufacturer#1|Brand#13|LARGE BRUSHED BRASS|1|LG CASE|902.00|lar accounts amo| +3|spring green yellow purple cornsilk|Manufacturer#4|Brand#42|STANDARD POLISHED BRASS|21|WRAP CASE|903.00|egular deposits hag| +4|cornflower chocolate smoke green pink|Manufacturer#3|Brand#34|SMALL PLATED BRASS|14|MED DRUM|904.00|p furiously r| +5|forest brown coral puff cream|Manufacturer#3|Brand#32|STANDARD POLISHED TIN|15|SM PKG|905.00| wake carefully | +6|bisque cornflower lawn forest magenta|Manufacturer#2|Brand#24|PROMO PLATED STEEL|4|MED BAG|906.00|sual a| +7|moccasin green thistle khaki floral|Manufacturer#1|Brand#11|SMALL PLATED COPPER|45|SM BAG|907.00|lyly. ex| +8|misty lace thistle snow royal|Manufacturer#4|Brand#44|PROMO BURNISHED TIN|41|LG DRUM|908.00|eposi| +9|thistle dim navajo dark gainsboro|Manufacturer#4|Brand#43|SMALL BURNISHED STEEL|12|WRAP CASE|909.00|ironic foxe| +10|linen pink saddle puff powder|Manufacturer#5|Brand#54|LARGE BURNISHED STEEL|44|LG CAN|910.01|ithely final deposit| diff --git a/rust/ballista/rust/scheduler/testdata/partsupp/partsupp.tbl b/rust/ballista/rust/scheduler/testdata/partsupp/partsupp.tbl new file mode 100644 index 0000000000000..45145385a1653 --- /dev/null +++ b/rust/ballista/rust/scheduler/testdata/partsupp/partsupp.tbl @@ -0,0 +1,10 @@ +1|2|3325|771.64|, even theodolites. regular, final theodolites eat after the carefully pending foxes. furiously regular deposits sleep slyly. carefully bold realms above the ironic dependencies haggle careful| +1|2502|8076|993.49|ven ideas. quickly even packages print. pending multipliers must have to are fluff| +1|5002|3956|337.09|after the fluffily ironic deposits? blithely special dependencies integrate furiously even excuses. blithely silent theodolites could have to haggle pending, express requests; fu| +1|7502|4069|357.84|al, regular dependencies serve carefully after the quickly final pinto beans. furiously even deposits sleep quickly final, silent pinto beans. fluffily reg| +2|3|8895|378.49|nic accounts. final accounts sleep furiously about the ironic, bold packages. regular, regular accounts| +2|2503|4969|915.27|ptotes. quickly pending dependencies integrate furiously. fluffily ironic ideas impress blithely above the express accounts. furiously even epitaphs need to wak| +2|5003|8539|438.37|blithely bold ideas. furiously stealthy packages sleep fluffily. slyly special deposits snooze furiously carefully regular accounts. regular deposits according to the accounts nag carefully slyl| +2|7503|3025|306.39|olites. deposits wake carefully. even, express requests cajole. carefully regular ex| +3|4|4651|920.92|ilent foxes affix furiously quickly unusual requests. even packages across the carefully even theodolites nag above the sp| +3|2504|4093|498.13|ending dependencies haggle fluffily. regular deposits boost quickly carefully regular requests. deposits affix furiously around the pinto beans. ironic, unusual platelets across the p| diff --git a/rust/ballista/rust/scheduler/testdata/region/region.tbl b/rust/ballista/rust/scheduler/testdata/region/region.tbl new file mode 100644 index 0000000000000..c5ebb63b621f6 --- /dev/null +++ b/rust/ballista/rust/scheduler/testdata/region/region.tbl @@ -0,0 +1,5 @@ +0|AFRICA|lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to | +1|AMERICA|hs use ironic, even requests. s| +2|ASIA|ges. thinly even pinto beans ca| +3|EUROPE|ly final courts cajole furiously final excuse| +4|MIDDLE EAST|uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl| diff --git a/rust/ballista/rust/scheduler/testdata/supplier/supplier.tbl b/rust/ballista/rust/scheduler/testdata/supplier/supplier.tbl new file mode 100644 index 0000000000000..d9c0e9f7e2016 --- /dev/null +++ b/rust/ballista/rust/scheduler/testdata/supplier/supplier.tbl @@ -0,0 +1,10 @@ +1|Supplier#000000001| N kD4on9OM Ipw3,gf0JBoQDd7tgrzrddZ|17|27-918-335-1736|5755.94|each slyly above the careful| +2|Supplier#000000002|89eJ5ksX3ImxJQBvxObC,|5|15-679-861-2259|4032.68| slyly bold instructions. idle dependen| +3|Supplier#000000003|q1,G3Pj6OjIuUYfUoH18BFTKP5aU9bEV3|1|11-383-516-1199|4192.40|blithely silent requests after the express dependencies are sl| +4|Supplier#000000004|Bk7ah4CK8SYQTepEmvMkkgMwg|15|25-843-787-7479|4641.08|riously even requests above the exp| +5|Supplier#000000005|Gcdm2rJRzl5qlTVzc|11|21-151-690-3663|-283.84|. slyly regular pinto bea| +6|Supplier#000000006|tQxuVm7s7CnK|14|24-696-997-4969|1365.79|final accounts. regular dolphins use against the furiously ironic decoys. | +7|Supplier#000000007|s,4TicNGB4uO6PaSqNBUq|23|33-990-965-2201|6820.35|s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit| +8|Supplier#000000008|9Sq4bBH2FQEmaFOocY45sRTxo6yuoG|17|27-498-742-3860|7627.85|al pinto beans. asymptotes haggl| +9|Supplier#000000009|1KhUgZegwM3ua7dsYmekYBsK|10|20-403-398-8662|5302.37|s. unusual, even requests along the furiously regular pac| +10|Supplier#000000010|Saygah3gYWMp72i PY|24|34-852-489-8585|3891.91|ing waters. regular requests ar|